You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "felipecrv (via GitHub)" <gi...@apache.org> on 2023/04/26 14:17:48 UTC

[GitHub] [arrow] felipecrv opened a new pull request, #35345: GH-35344: [C++] Draft an implementation of the LIST_VIEW array format

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

   ### Rationale for this change
   
   Mailing list discussion: https://lists.apache.org/thread/r28rw5n39jwtvn08oljl09d4q2c1ysvb
   
   ### What changes are included in this PR?
   
   Initial implementation of the new format in C++.
   
   ### Are these changes tested?
   
   Unit tests being written on every commit adding new functionality. More needs to be implemented for Integration Tests (required) to be implementable.
   
   ### Are there any user-facing changes?
   
   A new array format. It should have no impact for users that don't 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] github-actions[bot] commented on pull request #35345: GH-35344: [C++] Draft an implementation of the LIST_VIEW array format

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #35345:
URL: https://github.com/apache/arrow/pull/35345#issuecomment-1523502332

   * Closes: #35344


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/type_traits.cc:
##########
@@ -67,21 +67,23 @@ int RequiredValueAlignmentForBuffer(Type::type type_id, int buffer_index) {
     case Type::BINARY:  // Offsets may be cast to int32_t*
     case Type::DATE32:
     case Type::TIME32:
-    case Type::LIST:  // Offsets may be cast to int32_t*, data is in child array
-    case Type::MAP:   // This is a list array
+    case Type::LIST:       // Offsets may be cast to int32_t*, data is in child array
+    case Type::LIST_VIEW:  // Same as LIST

Review Comment:
   Fair enough. Made them all consistent, but dropped the comment about child values from the `LIST` 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   Waiting for @pitrou to take a look before I squash the fixups and rebase.


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -160,16 +166,69 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+void PutListViewOffsets(const Buffer& src, offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const BufferVector& buffers,

Review Comment:
   No. Since these are the list-view offsets.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -137,6 +137,77 @@ Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListArrayFromArray
   return std::make_shared<ArrayType>(std::move(data));
 }
 
+template <typename TYPE>
+Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListViewArrayFromArrays(
+    std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+    const Array& values, MemoryPool* pool, std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+    int64_t null_count = kUnknownNullCount) {
+  using offset_type = typename TYPE::offset_type;
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+  if (offsets.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List offsets must be ", OffsetArrowType::type_name());
+  }
+
+  if (sizes.length() != offsets.length() && sizes.length() != offsets.length() - 1) {
+    return Status::Invalid(
+        "List sizes must have the same length as offsets or one less than offsets");
+  }
+  if (sizes.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List sizes must be ", OffsetArrowType::type_name());
+  }
+
+  if (offsets.offset() != sizes.offset()) {
+    return Status::Invalid("List offsets and sizes must have the same offset");
+  }
+  const int64_t offset = sizes.offset();

Review Comment:
   Fair. I'm renaming it to `array_offset`.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   @pitrou PTAL when you can.


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/ipc/writer.cc:
##########
@@ -430,6 +492,36 @@ class RecordBatchSerializer {
     return Status::OK();
   }
 
+  template <typename T>
+  enable_if_list_view<typename T::TypeClass, Status> Visit(const T& array) {
+    using offset_type = typename T::offset_type;
+
+    offset_type min_offset = 0;
+    offset_type max_end = 0;
+    {
+      std::shared_ptr<Buffer> value_offsets;
+      RETURN_NOT_OK(
+          GetZeroBasedListViewOffsets<T>(array, &value_offsets, &min_offset, &max_end));
+      out_->body_buffers.push_back(std::move(value_offsets));
+    }
+    {
+      std::shared_ptr<Buffer> value_sizes;
+      RETURN_NOT_OK(GetListViewSizes<T>(array, &value_sizes));
+      out_->body_buffers.push_back(std::move(value_sizes));
+    }
+
+    --max_recursion_depth_;

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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }

Review Comment:
   Yeah I think this is to find the "longest `List` compatible mode". Thanks for so fast updating. I will go through it 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// value builder.
+  ///
+  /// \pre if is_valid is false, list_length MUST be 0
+  /// \param is_valid Whether the new list slot is valid
+  /// \param list_length The number of elements in the list
+  Status Append(bool is_valid, int64_t list_length) {

Review Comment:
   > This would be useful, can we add this in comment? Since the interface would be a bit tricky.
   
   This was done, but I forgot to confirm with a reply to your comment. :)



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_list_test.cc:
##########
@@ -434,16 +622,18 @@ class TestListArray : public ::testing::Test {
 
   void TestBulkAppendInvalid() {
     std::vector<int16_t> values = {0, 1, 2, 3, 4, 5, 6};
-    std::vector<int> lengths = {3, 0, 4};
     std::vector<uint8_t> is_valid = {1, 0, 1};
     // Should be {0, 3, 3} given the is_valid array

Review Comment:
   Is it OK if I delete it? (I pushed a fixup commit dong 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] felipecrv commented on pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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

   @bkietz I removed the unused `enum`, squashed the fixups, rebased, and force-pushed to get the latest commits from `main` and fix the conflict (rebase didn't even complain about 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {

Review Comment:
   I can extract a few more functions. Just a minute.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,170 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. TODO: describe the minimum validation
+  ///
+  /// TODO: review this

Review Comment:
   For me to review. I cleaned it up 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] bkietz commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -602,8 +691,11 @@ class ConcatenateImpl {
 }  // namespace
 
 Result<std::shared_ptr<Array>> Concatenate(const ArrayVector& arrays, MemoryPool* pool) {
-  if (arrays.size() == 0) {
-    return Status::Invalid("Must pass at least one array");
+  switch (arrays.size()) {
+    case 0:
+      return Status::Invalid("Must pass at least one array");
+    case 1:
+      return arrays[0];

Review Comment:
   https://github.com/apache/arrow/issues/37878



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/type.h:
##########
@@ -990,6 +990,71 @@ class ARROW_EXPORT LargeListType : public BaseListType {
   std::string ComputeFingerprint() const override;
 };
 
+/// \brief Type class for array of list views
+class ARROW_EXPORT ListViewType : public BaseListType {
+ public:
+  static constexpr Type::type type_id = Type::LIST_VIEW;
+  using offset_type = int32_t;
+
+  static constexpr const char* type_name() { return "list_view"; }
+
+  // ListView can contain any other logical value type
+  explicit ListViewType(const std::shared_ptr<DataType>& value_type)

Review Comment:
   I created an issue for this problem: https://github.com/apache/arrow/issues/37891



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/compute/kernels/scalar_if_else_benchmark.cc:
##########
@@ -67,11 +67,15 @@ struct GetBytesProcessedVisitor {
   }
 
   template <typename ArrowType>
-  enable_if_var_size_list<ArrowType, Status> Visit(const ArrowType& type) {
+  enable_if_var_length_list_like<ArrowType, Status> Visit(const ArrowType& type) {
     using ArrayType = typename TypeTraits<ArrowType>::ArrayType;
     using OffsetType = typename TypeTraits<ArrowType>::OffsetType::c_type;
 
-    total_bytes += (arr->length() + 1) * sizeof(OffsetType);
+    const auto num_offsets = is_list_view(type) ? arr->length() : arr->length() + 1;
+    total_bytes += num_offsets * sizeof(OffsetType);
+    // NOTE: the sizes buffer is not counted when type is a list-view as that
+    // can make the throughput numbers look better just because the sizes add
+    // increase the number of bytes in the input.

Review Comment:
   I'm fixing a typo here: ~add~ increase



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   @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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -797,57 +811,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > ",
+                             offset_limit);
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    for (int64_t i = 0; i < data.length; ++i) {
+      const auto size = sizes[i];
+      if (size >= 0) {
+        const auto offset = offsets[i];
+        if (offset < 0 || offset > offset_limit) {
+          return OutOfBoundsListViewOffset<offset_type>(i, offset_limit);
+        }
+        if (size > offset_limit - offset) {
+          return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+        }
+      } else {
+        return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
-    if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+    const bool non_empty = data.length > 0;
+    if constexpr (is_list_view) {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return Status::Invalid("offsets buffer is null");
+      }
+      if (!IsBufferValid(2)) {
+        return Status::Invalid("sizes buffer is null");
+      }
+    } else {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return non_empty ? Status::Invalid("Non-empty array but offsets are null")
+                         : Status::OK();
       }
-      return Status::OK();
     }
 
-    // An empty list array can have 0 offsets
     const auto offsets_byte_size = data.buffers[1]->size();
     const auto required_offsets = ((data.length > 0) || (offsets_byte_size > 0))
-                                      ? data.length + data.offset + 1
+                                      ? data.length + data.offset + (is_list_view ? 0 : 1)
                                       : 0;
     if (offsets_byte_size / static_cast<int32_t>(sizeof(offset_type)) <
         required_offsets) {
       return Status::Invalid("Offsets buffer size (bytes): ", offsets_byte_size,
                              " isn't large enough for length: ", data.length,
                              " and offset: ", data.offset);
     }
+    if constexpr (is_list_view) {
+      const auto required_sizes = data.length + data.offset;
+      const auto sizes_bytes_size = data.buffers[2]->size();
+      if (sizes_bytes_size / static_cast<int32_t>(sizeof(offset_type)) < required_sizes) {
+        return Status::Invalid("Sizes buffer size (bytes): ", sizes_bytes_size,
+                               " isn't large enough for length: ", data.length,
+                               " and offset: ", data.offset);
+      }
+    }
 
     if (full_validation && required_offsets > 0) {
-      // Validate all offset values
-      const offset_type* offsets = data.GetValues<offset_type>(1);
-
-      auto prev_offset = offsets[0];
-      if (prev_offset < 0) {
-        return Status::Invalid(
-            "Offset invariant failure: array starts at negative offset ", prev_offset);
-      }
-      for (int64_t i = 1; i <= data.length; ++i) {
-        const auto current_offset = offsets[i];
-        if (current_offset < prev_offset) {
-          return Status::Invalid(
-              "Offset invariant failure: non-monotonic offset at slot ", i, ": ",
-              current_offset, " < ", prev_offset);
-        }
-        if (current_offset > offset_limit) {
-          return Status::Invalid("Offset invariant failure: offset for slot ", i,
-                                 " out of bounds: ", current_offset, " > ", offset_limit);
-        }
-        prev_offset = current_offset;
+      if constexpr (is_list_view) {
+        return FullyValidateOffsetsAndSizes<offset_type>(offset_limit);
+      } else {
+        return FullyValidateOffsets<offset_type>(offset_limit);
       }
     }
     return Status::OK();
   }
 
+ public:
+  template <typename TypeClass>
+  enable_if_list_view<TypeClass, Status> ValidateOffsetsAndSizes(const TypeClass& type,
+                                                                 int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }
+
+  template <typename TypeClass>
+  std::enable_if_t<is_var_length_list_type<TypeClass>::value ||
+                       is_base_binary_like(TypeClass::type_id),
+                   Status>
+  ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }

Review Comment:
   Two reasons:
    - Provide a simple public interface to the class
    - Constrain the types that can be passed as template parameters while all the internal private functions can assume the passed types are valid



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,172 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.
+  ///
+  /// \param[in] offsets An array of int32 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int32 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the list-views in this array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> offsets() const;
+
+  /// \brief Return list-view sizes as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> sizes() const;
+
+ protected:
+  // This constructor defers SetData to a derived array class
+  ListViewArray() = default;
+
+  void SetData(const std::shared_ptr<ArrayData>& data);
+};
+
+/// \brief Concrete Array class for large list-view data (with 64-bit offsets
+/// and sizes)
+class ARROW_EXPORT LargeListViewArray : public BaseListViewArray<LargeListViewType> {
+ public:
+  explicit LargeListViewArray(std::shared_ptr<ArrayData> data);
+
+  LargeListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                     std::shared_ptr<Buffer> value_offsets,
+                     std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                     std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                     int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct LargeListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an LargeListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. TODO: describe the minimum validation

Review Comment:
   There were two copies of this. Removing the second now. Pushing 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   I'm a bit busy this two days(And actually I'm not so familiar with these part of code TAT) . I'll take a look carefully this weekend :-)


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  auto is_null_or_empty = [&](int64_t i) {
+    return (validity && !bit_util::GetBit(validity, list_view_array.offset() + i)) ||
+           sizes[i] == 0;
+  };
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid, non-empty list-view and last offset
+  // of the current contiguous fragment in values.
+  constexpr int64_t kUninitialized = -1;
+  int64_t first_i = kUninitialized;
+  offset_type end_offset;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    if (offsets[i] == end_offset) {
+      end_offset += sizes[i];
+      continue;
+    }
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+  }
+  if (first_i != kUninitialized) {
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+  }
+
+  // Final attempt to avoid invoking Concatenate().
+  if (non_null_fragments.size() == 1) {
+    return non_null_fragments[0];
+  } else if (non_null_fragments.size() == 0) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  return Concatenate(non_null_fragments, memory_pool);
+}
+
 std::shared_ptr<Array> BoxOffsets(const std::shared_ptr<DataType>& boxed_type,
                                   const ArrayData& data) {
+  const int64_t num_offsets =
+      is_list_view(data.type->id()) ? data.length : data.length + 1;
   std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[1]};
   auto offsets_data =
-      std::make_shared<ArrayData>(boxed_type, data.length + 1, std::move(buffers),
+      std::make_shared<ArrayData>(boxed_type, /*length=*/num_offsets, std::move(buffers),
+                                  /*null_count=*/0, data.offset);
+  return MakeArray(offsets_data);
+}
+
+std::shared_ptr<Array> BoxSizes(const std::shared_ptr<DataType>& boxed_type,
+                                const ArrayData& data) {
+  DCHECK(is_list_view(data.type->id()));
+  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[2]};
+  auto offsets_data =

Review Comment:
   Hmm I also say `SwapOffsets` in latter code, maybe offset is also ok 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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -113,33 +122,30 @@ Status ConcatenateOffsets(const BufferVector& buffers, MemoryPool* pool,
   values_ranges->resize(buffers.size());
 
   // allocate output buffer
-  int64_t out_length = 0;
-  for (const auto& buffer : buffers) {
-    out_length += buffer->size() / sizeof(Offset);
-  }
-  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer((out_length + 1) * sizeof(Offset), pool));
-  auto dst = reinterpret_cast<Offset*>((*out)->mutable_data());
+  const int64_t out_size_in_bytes = SumBufferSizesInBytes(buffers);
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(sizeof(Offset) + out_size_in_bytes, pool));

Review Comment:
   Yes. This is used only for lists. Not list-views and lists need one extra offset.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/ipc/writer.cc:
##########
@@ -350,6 +350,67 @@ class RecordBatchSerializer {
     return Status::OK();
   }
 
+  template <typename ArrayType, typename offset_type = typename ArrayType::offset_type>
+  Status GetZeroBasedListViewOffsets(const ArrayType& array,
+                                     std::shared_ptr<Buffer>* out_value_offsets,
+                                     offset_type* out_min_offset,
+                                     offset_type* out_max_end) {
+    auto offsets = array.value_offsets();
+    auto sizes = array.value_sizes();
+
+    const int64_t required_bytes = sizeof(offset_type) * array.length();
+    if (array.offset() != 0) {
+      // If we have a non-zero offset, it's likely that the smallest offset is
+      // not zero. We must a) create a new offsets array with shifted offsets and
+      // b) slice the values array accordingly.
+
+      ARROW_ASSIGN_OR_RAISE(auto shifted_offsets,
+                            AllocateBuffer(required_bytes, options_.memory_pool));
+      offset_type min_offset = 0;
+      offset_type max_end = 0;
+      if (array.length() > 0) {
+        min_offset = std::numeric_limits<offset_type>::max();
+        for (int i = 0; i < array.length(); ++i) {
+          min_offset = std::min(min_offset, array.value_offset(i));
+          max_end = std::max(max_end, array.value_offset(i) + array.value_length(i));
+        }
+      }
+
+      auto* dest_offsets = shifted_offsets->mutable_data_as<offset_type>();
+
+      for (int i = 0; i < array.length(); ++i) {
+        dest_offsets[i] = array.value_offset(i) - min_offset;
+      }
+      *out_min_offset = min_offset;
+      *out_max_end = max_end;
+      offsets = std::move(shifted_offsets);
+    } else {
+      // ARROW-6046: Slice offsets to used extent, in case we have a truncated
+      // slice
+      if (offsets != nullptr && offsets->size() > required_bytes) {
+        offsets = SliceBuffer(offsets, 0, required_bytes);
+      }
+      *out_min_offset = 0;
+      *out_max_end = static_cast<offset_type>(array.values()->length());
+    }
+    *out_value_offsets = std::move(offsets);
+    return Status::OK();
+  }
+
+  template <typename ArrayType, typename offset_type = typename ArrayType::offset_type>
+  Status GetListViewSizes(const ArrayType& array,
+                          std::shared_ptr<Buffer>* out_value_sizes) {
+    const int64_t required_bytes = sizeof(offset_type) * array.length();
+    auto sizes = array.value_sizes();
+    if (sizes != nullptr && (array.offset() != 0 || sizes->size() > required_bytes)) {

Review Comment:
   Remember that in the beginning I was open to list-views having null buffers just like empty lists can have empty buffers? This is the reason. And I think keeping that check would be wise in case we revert that decision in the future.
   
   This extra condition avoids an extra  `shared_ptr` allocation. Cheap, but something worth avoiding I guess. This was based on the code from `GetZeroBasedValueOffsets` introduced in 7f6e6ab2.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }

Review Comment:
   I'm removing it. Unlike the `ree_util::ValuesArray` , I haven't used 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -203,6 +203,54 @@ TEST_F(ConcatenateTest, LargeListType) {
   });
 }
 
+TEST_F(ConcatenateTest, ListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    auto values_size = size * 4;
+    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
+
+    std::shared_ptr<Array> offsets;
+    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    offsets_vector.front() = 0;
+    offsets_vector.back() = values_size;
+    ArrayFromVector<Int32Type>(offsets_vector, &offsets);
+
+    std::shared_ptr<Array> sizes;
+    std::vector<int32_t> sizes_vector;
+    sizes_vector.reserve(size);
+    for (int32_t i = 0; i < size; ++i) {
+      sizes_vector.push_back(offsets_vector[i + 1] - offsets_vector[i]);
+      ASSERT_LE(offsets_vector[i] + sizes_vector.back(), values_size);
+    }
+    ASSERT_EQ(offsets_vector.size(), sizes_vector.size() + 1);
+    ArrayFromVector<Int32Type>(sizes_vector, &sizes);
+
+    ASSERT_OK_AND_ASSIGN(*out, ListViewArray::FromArrays(*offsets, *sizes, *values));
+    ASSERT_OK((**out).ValidateFull());
+  });
+}
+
+TEST_F(ConcatenateTest, LargeListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {

Review Comment:
   New commit added de-duplicating for both lists and list-views.



-- 
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 #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,62 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }
+
+namespace internal {
+
+/// \brief Calculate the smallest continuous range of values used by the
+/// var-length list-like input (list, map and list-view types).
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return A pair of (offset, length) describing the range
+ARROW_EXPORT Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(
+    const ArraySpan& input);
+
+/// \brief Builds a ListViewArray from a ListArray

Review Comment:
   Nit: infinitive (also below)
   ```suggestion
   /// \brief Build a ListViewArray from a ListArray
   ```



##########
cpp/src/arrow/testing/random.cc:
##########
@@ -623,6 +739,31 @@ std::shared_ptr<Array> RandomArrayGenerator::List(const Array& values, int64_t s
   return *::arrow::ListArray::FromArrays(*offsets, values);
 }
 
+std::shared_ptr<Array> RandomArrayGenerator::ListView(
+    const Array& values, int64_t size, double null_probability, bool force_empty_nulls,
+    bool zero_undefined_offsets, int64_t alignment, MemoryPool* memory_pool) {
+  using offset_type = int32_t;
+  using OffsetArrayType = Int32Array;
+  using OffsetArrowType = Int32Type;
+
+  DCHECK_LE(values.length(), std::numeric_limits<offset_type>::max());
+  DCHECK_LE(size, std::numeric_limits<offset_type>::max());
+  const auto values_length = static_cast<offset_type>(values.length());
+
+  const offset_type avg_length = (values_length - 1) / static_cast<offset_type>(size) + 1;
+  const offset_type min_length = 0;
+  const offset_type max_length = std::min(std::max(2 * avg_length, 1), values_length);
+  const auto lengths = internal::checked_pointer_cast<OffsetArrayType>(
+      Numeric<OffsetArrowType, offset_type>(size, min_length, max_length,
+                                            null_probability));
+
+  const auto offsets = ViewOffsetsFromLengthsArray<OffsetArrayType, offset_type>(
+      seed(), avg_length, values_length, *lengths, force_empty_nulls,
+      zero_undefined_offsets, alignment, memory_pool);

Review Comment:
   I'm not sure I understand the algorithm here, and it would deserve explaining in a comment.
   
   Intuitively, this is what I'd do:
   1. accept a coverage percentage as a parameter (for example, 90% of the child array would be covered by the list views array)
   2. randomly generate the offsets as in `RandomArrayGenerator::List`
   3. generate the sizes so as to roughly satisfy the coverage parameter (that is, for each `offset[i]`, generate a size that's on average equal to `coverage * (offset[i+1] - offset[i])`)
   
   Also perhaps randomly shuffle the (offset, size) pairs.



##########
cpp/src/arrow/scalar_test.cc:
##########
@@ -394,6 +394,10 @@ class TestRealScalar : public ::testing::Test {
 
   void TestLargeListOf() { TestListOf<LargeListScalar>(large_list(type_)); }
 
+  void TestListViewOf() { TestListOf<ListViewScalar>(list_view(type_)); }
+
+  void TestLargeListViewOf() { TestListOf<LargeListViewScalar>(large_list_view(type_)); }

Review Comment:
   Do these tests implicitly exercise `MakeNullScalar`? If not, can you add tests for it?



##########
cpp/src/arrow/compute/kernels/scalar_if_else_benchmark.cc:
##########
@@ -67,11 +67,15 @@ struct GetBytesProcessedVisitor {
   }
 
   template <typename ArrowType>
-  enable_if_var_size_list<ArrowType, Status> Visit(const ArrowType& type) {
+  enable_if_var_length_list_like<ArrowType, Status> Visit(const ArrowType& type) {
     using ArrayType = typename TypeTraits<ArrowType>::ArrayType;
     using OffsetType = typename TypeTraits<ArrowType>::OffsetType::c_type;
 
-    total_bytes += (arr->length() + 1) * sizeof(OffsetType);
+    const auto num_offsets = is_list_view(type) ? arr->length() : arr->length() + 1;
+    total_bytes += num_offsets * sizeof(OffsetType);
+    // NOTE: the sizes buffer is not counted when type is a list-view as that
+    // can make the throughput numbers look better just because the sizes add
+    // increase the number of bytes in the input.

Review Comment:
   Right. It's also why, in general, items/sec is a much more meaningful number to look at (a more memory-efficient representation could get artificially lower "performance" when measured in bytes/sec).



##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -203,6 +203,54 @@ TEST_F(ConcatenateTest, LargeListType) {
   });
 }
 
+TEST_F(ConcatenateTest, ListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    auto values_size = size * 4;
+    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
+
+    std::shared_ptr<Array> offsets;
+    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    offsets_vector.front() = 0;
+    offsets_vector.back() = values_size;
+    ArrayFromVector<Int32Type>(offsets_vector, &offsets);
+
+    std::shared_ptr<Array> sizes;
+    std::vector<int32_t> sizes_vector;
+    sizes_vector.reserve(size);
+    for (int32_t i = 0; i < size; ++i) {
+      sizes_vector.push_back(offsets_vector[i + 1] - offsets_vector[i]);
+      ASSERT_LE(offsets_vector[i] + sizes_vector.back(), values_size);
+    }
+    ASSERT_EQ(offsets_vector.size(), sizes_vector.size() + 1);
+    ArrayFromVector<Int32Type>(sizes_vector, &sizes);
+
+    ASSERT_OK_AND_ASSIGN(*out, ListViewArray::FromArrays(*offsets, *sizes, *values));
+    ASSERT_OK((**out).ValidateFull());
+  });
+}
+
+TEST_F(ConcatenateTest, LargeListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {

Review Comment:
   Can we perhaps cut down on the code repetition and factor this out to share the same code between ListView and LargeListView?



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,170 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. TODO: describe the minimum validation
+  ///
+  /// TODO: review this

Review Comment:
   Is the TODO for you to act on? Or does it require others' opinions?



##########
cpp/src/arrow/testing/random.h:
##########
@@ -438,6 +438,23 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
                               int64_t alignment = kDefaultBufferAlignment,
                               MemoryPool* memory_pool = default_memory_pool());
 
+  /// \brief Generate a random ListViewArray
+  ///
+  /// \param[in] values The underlying values array
+  /// \param[in] size The size of the generated list array
+  /// \param[in] null_probability the probability of a list value being null
+  /// \param[in] force_empty_nulls if true, null list entries must have 0 length

Review Comment:
   Need to add a doc for the `zero_undefined_offsets` parameter.



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -43,30 +43,31 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// ListArray
+// VarLengthListLikeArray
 
 template <typename TYPE>
-class BaseListArray;
+class VarLengthListLikeArray;

Review Comment:
   Keep a `Base` prefix here as well?



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, input.offset, /*length=*/i + 1, [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; ++j) {
+          const auto offset = static_cast<int64_t>(offsets[input.offset + i + j]);
+          const auto size = sizes[input.offset + i + j];
+          if (size > 0) {
+            if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+              if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+                // 64-bit overflow detected. This is not possible on a valid list-view,
+                // but we saturate max_end to the maximum possible value to avoid
+                // undefined behavior.
+                max_end = kInt64Max;
+                return;
+              }
+            }
+            max_end = std::max(max_end, offset + size);
+          }
+        }
+      });
+  return max_end;
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByListView(const ArraySpan& input) {
+  DCHECK(is_list_view(*input.type));
+  if (input.length == 0 || input.GetNullCount() == input.length) {
+    return {0, 0};
+  }
+  const int64_t min_offset = MinViewOffset<offset_type>(input);
+  const int64_t max_end = MaxViewEnd<offset_type>(input);
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByList(const ArraySpan& input) {
+  DCHECK(is_var_length_list(*input.type));
+  if (input.length == 0) {
+    return {0, 0};
+  }
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const int64_t min_offset = offsets[input.offset];
+  const int64_t max_end = offsets[input.offset + input.length];
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename DestListViewType, typename SrcListType>
+Result<std::shared_ptr<ArrayData>> ListViewFromListImpl(
+    const std::shared_ptr<ArrayData>& list_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListType::offset_type,
+                   typename DestListViewType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename SrcListType::offset_type;
+  const auto& list_type = checked_cast<const SrcListType&>(*list_data->type);
+
+  // To re-use the validity and offsets buffers, a sizes buffer with enough
+  // padding on the beginning is allocated and filled with the sizes after
+  // list_data->offset.
+  const int64_t buffer_length = list_data->offset + list_data->length;
+  ARROW_ASSIGN_OR_RAISE(auto sizes_buffer,
+                        AllocateBuffer(buffer_length * sizeof(offset_type), pool));
+  const auto* offsets = list_data->template GetValues<offset_type>(1, 0);
+  auto* sizes = reinterpret_cast<offset_type*>(sizes_buffer->mutable_data());
+  for (int64_t i = list_data->offset; i < buffer_length; i++) {
+    sizes[i] = offsets[i + 1] - offsets[i];
+  }
+  BufferVector buffers = {list_data->buffers[0], list_data->buffers[1],
+                          std::move(sizes_buffer)};
+
+  return ArrayData::Make(std::make_shared<DestListViewType>(list_type.value_type()),
+                         list_data->length, std::move(buffers),
+                         {list_data->child_data[0]}, list_data->null_count,
+                         list_data->offset);
+}
+
+template <typename DestListType, typename SrcListViewType>
+Result<std::shared_ptr<ArrayData>> ListFromListViewImpl(
+    const std::shared_ptr<ArrayData>& list_view_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListViewType::offset_type,
+                   typename DestListType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename DestListType::offset_type;
+  using ListBuilderType = typename TypeTraits<DestListType>::BuilderType;
+
+  const auto& list_view_type =
+      checked_cast<const SrcListViewType&>(*list_view_data->type);
+  const auto& value_type = list_view_type.value_type();
+  const auto list_type = std::make_shared<DestListType>(value_type);
+
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<ArrayBuilder> value_builder,
+                        MakeBuilder(value_type, pool));
+  auto list_builder = std::make_shared<ListBuilderType>(pool, value_builder, list_type);
+  RETURN_NOT_OK(list_builder->Reserve(list_view_data->length));
+
+  ArraySpan values{*list_view_data->child_data[0]};
+  const auto* in_validity_bitmap = list_view_data->GetValues<uint8_t>(0);
+  const auto* in_offsets = list_view_data->GetValues<offset_type>(1);
+  const auto* in_sizes = list_view_data->GetValues<offset_type>(2);
+  for (int64_t i = 0; i < list_view_data->length; ++i) {
+    const bool is_valid =
+        !in_validity_bitmap ||
+        bit_util::GetBit(in_validity_bitmap, list_view_data->offset + i);
+    const int64_t size = is_valid ? in_sizes[i] : 0;
+    RETURN_NOT_OK(list_builder->Append(is_valid, size));
+    RETURN_NOT_OK(value_builder->AppendArraySlice(values, in_offsets[i], size));
+  }
+  std::shared_ptr<ArrayData> list_array_data;
+  RETURN_NOT_OK(list_builder->FinishInternal(&list_array_data));
+  return list_array_data;
+}
+
+}  // namespace
+
+Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(const ArraySpan& input) {
+  switch (input.type->id()) {
+    case Type::LIST:
+      return RangeOfValuesUsedByList<ListType::offset_type>(input);
+    case Type::MAP:
+      return RangeOfValuesUsedByList<MapType::offset_type>(input);
+    case Type::LARGE_LIST:
+      return RangeOfValuesUsedByList<LargeListType::offset_type>(input);
+    case Type::LIST_VIEW:
+      return RangeOfValuesUsedByListView<ListViewType::offset_type>(input);
+    case Type::LARGE_LIST_VIEW:
+      return RangeOfValuesUsedByListView<LargeListViewType::offset_type>(input);
+    default:
+      break;
+  }
+  DCHECK(!is_var_length_list_like(*input.type));
+  return Status::Invalid("RangeOfValuesUsed: input is not a var-length list-like array");

Review Comment:
   Make this `Status::TypeError` since it's a type error.



##########
cpp/src/arrow/util/list_util_test.cc:
##########
@@ -0,0 +1,233 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array/builder_nested.h"
+#include "arrow/util/list_util.h"
+
+#include "arrow/testing/builder.h"
+#include "arrow/testing/gtest_util.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+using ListAndListViewTypes =
+    ::testing::Types<ListType, LargeListType, ListViewType, LargeListViewType>;
+
+template <typename T>
+class TestListUtils : public ::testing::Test {
+ public:
+  using TypeClass = T;
+  using offset_type = typename TypeClass::offset_type;
+  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
+
+  void SetUp() override {
+    value_type_ = int16();
+    type_ = std::make_shared<T>(value_type_);
+
+    std::unique_ptr<ArrayBuilder> tmp;
+    ASSERT_OK(MakeBuilder(pool_, type_, &tmp));
+    builder_.reset(checked_cast<BuilderType*>(tmp.release()));
+  }
+
+  void TestRangeOfValuesUsed() {
+    std::shared_ptr<ArrayData> result;
+
+    // These list-views are built manually with the list-view builders instead
+    // of using something like ArrayFromJSON() because we want to test the
+    // RangeOfValuesUsed() function's ability to handle arrays containing
+    // overlapping list-views.

Review Comment:
   Or you can use `ListViewArray::FromArrays` to achieve the same effect :-)



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, input.offset, /*length=*/i + 1, [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; ++j) {
+          const auto offset = static_cast<int64_t>(offsets[input.offset + i + j]);
+          const auto size = sizes[input.offset + i + j];
+          if (size > 0) {
+            if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+              if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+                // 64-bit overflow detected. This is not possible on a valid list-view,
+                // but we saturate max_end to the maximum possible value to avoid
+                // undefined behavior.
+                max_end = kInt64Max;
+                return;
+              }
+            }
+            max_end = std::max(max_end, offset + size);
+          }
+        }
+      });
+  return max_end;
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByListView(const ArraySpan& input) {
+  DCHECK(is_list_view(*input.type));
+  if (input.length == 0 || input.GetNullCount() == input.length) {
+    return {0, 0};
+  }
+  const int64_t min_offset = MinViewOffset<offset_type>(input);
+  const int64_t max_end = MaxViewEnd<offset_type>(input);
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByList(const ArraySpan& input) {
+  DCHECK(is_var_length_list(*input.type));
+  if (input.length == 0) {
+    return {0, 0};
+  }
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const int64_t min_offset = offsets[input.offset];
+  const int64_t max_end = offsets[input.offset + input.length];
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename DestListViewType, typename SrcListType>
+Result<std::shared_ptr<ArrayData>> ListViewFromListImpl(
+    const std::shared_ptr<ArrayData>& list_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListType::offset_type,
+                   typename DestListViewType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename SrcListType::offset_type;
+  const auto& list_type = checked_cast<const SrcListType&>(*list_data->type);
+
+  // To re-use the validity and offsets buffers, a sizes buffer with enough
+  // padding on the beginning is allocated and filled with the sizes after
+  // list_data->offset.
+  const int64_t buffer_length = list_data->offset + list_data->length;
+  ARROW_ASSIGN_OR_RAISE(auto sizes_buffer,
+                        AllocateBuffer(buffer_length * sizeof(offset_type), pool));
+  const auto* offsets = list_data->template GetValues<offset_type>(1, 0);
+  auto* sizes = reinterpret_cast<offset_type*>(sizes_buffer->mutable_data());
+  for (int64_t i = list_data->offset; i < buffer_length; i++) {
+    sizes[i] = offsets[i + 1] - offsets[i];
+  }
+  BufferVector buffers = {list_data->buffers[0], list_data->buffers[1],
+                          std::move(sizes_buffer)};
+
+  return ArrayData::Make(std::make_shared<DestListViewType>(list_type.value_type()),
+                         list_data->length, std::move(buffers),
+                         {list_data->child_data[0]}, list_data->null_count,
+                         list_data->offset);
+}
+
+template <typename DestListType, typename SrcListViewType>
+Result<std::shared_ptr<ArrayData>> ListFromListViewImpl(
+    const std::shared_ptr<ArrayData>& list_view_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListViewType::offset_type,
+                   typename DestListType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename DestListType::offset_type;
+  using ListBuilderType = typename TypeTraits<DestListType>::BuilderType;
+
+  const auto& list_view_type =
+      checked_cast<const SrcListViewType&>(*list_view_data->type);
+  const auto& value_type = list_view_type.value_type();
+  const auto list_type = std::make_shared<DestListType>(value_type);
+
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<ArrayBuilder> value_builder,
+                        MakeBuilder(value_type, pool));
+  auto list_builder = std::make_shared<ListBuilderType>(pool, value_builder, list_type);
+  RETURN_NOT_OK(list_builder->Reserve(list_view_data->length));

Review Comment:
   Is there a function to get the logical values length of a list view array? If so, we could also call `value_builder->Reserve`.



##########
cpp/src/arrow/ipc/writer.cc:
##########
@@ -430,6 +430,14 @@ class RecordBatchSerializer {
     return Status::OK();
   }
 
+  Status Visit(const ListViewArray& array) {
+    return Status::NotImplemented("list-view array in IPC");
+  }
+
+  Status Visit(const LargeListViewArray& array) {
+    return Status::NotImplemented("large list-view array in IPC");
+  }

Review Comment:
   IPC support will have to be addressed very soon as it's integral to the Arrow specification.



##########
cpp/src/arrow/ipc/json_simple.cc:
##########
@@ -555,8 +563,9 @@ class ListConverter final : public ConcreteConverter<ListConverter<TYPE>> {
     if (json_obj.IsNull()) {
       return this->AppendNull();
     }
-    RETURN_NOT_OK(builder_->Append());
     // Extend the child converter with this JSON array
+    ARROW_ASSIGN_OR_RAISE(auto size, this->SizeOfJSONArray(json_obj));
+    RETURN_NOT_OK(builder_->Append(true, size));

Review Comment:
   Hmm, this looks... confusing? Given other APIs, this could mean something like "append `size` non-nulls element"?



##########
cpp/src/arrow/type.h:
##########
@@ -990,6 +990,71 @@ class ARROW_EXPORT LargeListType : public BaseListType {
   std::string ComputeFingerprint() const override;
 };
 
+/// \brief Type class for array of list views
+class ARROW_EXPORT ListViewType : public BaseListType {
+ public:
+  static constexpr Type::type type_id = Type::LIST_VIEW;
+  using offset_type = int32_t;
+
+  static constexpr const char* type_name() { return "list_view"; }
+
+  // ListView can contain any other logical value type
+  explicit ListViewType(const std::shared_ptr<DataType>& value_type)

Review Comment:
   Should this constructor and the others below take their `std::shared_ptr` argument by value?



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types

Review Comment:
   Well, not really. For list views you must pass the actual list view size, while for lists you can pass whatever you want :-)



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be

Review Comment:
   ```suggestion
       // offsets are assumed to be valid, but the first length-1 sizes have to be
   ```



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -248,6 +305,38 @@ class ConcatenateImpl {
     return ConcatenateImpl(child_data, pool_).Concatenate(&out_->child_data[0]);
   }
 
+  template <typename T>
+  enable_if_list_view<T, Status> Visit(const T& type) {
+    using offset_type = typename T::offset_type;
+    out_->buffers.resize(3);
+    out_->child_data.resize(1);
+
+    // Calculate the ranges of values that each list-view array uses
+    std::vector<Range> value_ranges;
+    value_ranges.reserve(in_.size());
+    for (const auto& input : in_) {
+      ArraySpan input_span(*input);
+      Range range;
+      ARROW_ASSIGN_OR_RAISE(std::tie(range.offset, range.length),
+                            list_util::internal::RangeOfValuesUsed(input_span));
+      value_ranges.push_back(range);

Review Comment:
   Or perhaps `Range` should simply allow construction from a `std::pair<int64_t, int64_t>`?
   
   That said, unadorned pairs or tuples are not particularly pretty. It's a pity C++ doesn't have something like Python's `namedtuple`.



##########
cpp/src/arrow/builder.cc:
##########
@@ -219,6 +219,20 @@ struct MakeBuilderImpl {
     return Status::OK();
   }
 
+  Status Visit(const ListViewType& list_view_type) {
+    std::shared_ptr<DataType> value_type = list_view_type.value_type();
+    ARROW_ASSIGN_OR_RAISE(auto value_builder, ChildBuilder(value_type));
+    out.reset(new ListViewBuilder(pool, std::move(value_builder), std::move(type)));
+    return Status::OK();
+  }
+
+  Status Visit(const LargeListViewType& large_list_view_type) {
+    std::shared_ptr<DataType> value_type = large_list_view_type.value_type();
+    ARROW_ASSIGN_OR_RAISE(auto value_builder, ChildBuilder(value_type));
+    out.reset(new LargeListViewBuilder(pool, std::move(value_builder), type));

Review Comment:
   ```suggestion
       out.reset(new LargeListViewBuilder(pool, std::move(value_builder), std::move(type)));
   ```



##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -203,6 +203,54 @@ TEST_F(ConcatenateTest, LargeListType) {
   });
 }
 
+TEST_F(ConcatenateTest, ListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    auto values_size = size * 4;
+    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
+
+    std::shared_ptr<Array> offsets;
+    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    offsets_vector.front() = 0;
+    offsets_vector.back() = values_size;
+    ArrayFromVector<Int32Type>(offsets_vector, &offsets);
+
+    std::shared_ptr<Array> sizes;
+    std::vector<int32_t> sizes_vector;
+    sizes_vector.reserve(size);
+    for (int32_t i = 0; i < size; ++i) {
+      sizes_vector.push_back(offsets_vector[i + 1] - offsets_vector[i]);

Review Comment:
   Can we make things more interesting by making the list views non-contiguous?
   For example:
   ```suggestion
         sizes_vector.push_back((offsets_vector[i + 1] - offsets_vector[i]) * 2 / 3);
   ```



##########
cpp/src/arrow/integration/json_internal.cc:
##########
@@ -1492,6 +1506,14 @@ class ArrayReader {
     return CreateList<T>(type_);
   }
 
+  Status Visit(const ListViewType& type) {
+    return Status::NotImplemented("list-view in JSON");
+  }
+
+  Status Visit(const LargeListViewType& type) {
+    return Status::NotImplemented("large list-view in JSON");
+  }

Review Comment:
   Similarly, this will need addressing very soon for proper integration testing between at least C++ and Go.



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -602,8 +691,11 @@ class ConcatenateImpl {
 }  // namespace
 
 Result<std::shared_ptr<Array>> Concatenate(const ArrayVector& arrays, MemoryPool* pool) {
-  if (arrays.size() == 0) {
-    return Status::Invalid("Must pass at least one array");
+  switch (arrays.size()) {
+    case 0:
+      return Status::Invalid("Must pass at least one array");
+    case 1:
+      return arrays[0];

Review Comment:
   We shouldn't do that, because we've been recommending `Concatenate` as a way of materializing a slice to trim down memory use.
   
   We might even want to add a test for that, actually, to make sure that idiom continues working.



##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -203,6 +203,54 @@ TEST_F(ConcatenateTest, LargeListType) {
   });
 }
 
+TEST_F(ConcatenateTest, ListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    auto values_size = size * 4;
+    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
+
+    std::shared_ptr<Array> offsets;
+    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    offsets_vector.front() = 0;
+    offsets_vector.back() = values_size;

Review Comment:
   This isn't needed for list views, is it?
   



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// value builder.
+  ///
+  /// \pre if is_valid is false, list_length MUST be 0
+  /// \param is_valid Whether the new list slot is valid
+  /// \param list_length The number of elements in the list
+  Status Append(bool is_valid, int64_t list_length) {

Review Comment:
   Is it really necessary to have the caller pass the list length, or could it simply inferred in the next `Append` or `Finish` call?



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,173 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > offset_limit");
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+              }
+              if (size > offset_limit - offset) {
+                return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+              }
+            } else if (size < 0) {
+              return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+            }
+          }
+        }
+      }
+    } else {
+      for (; slot < data.length; slot++) {
+        const auto size = sizes[slot];
+        if (size > 0) {
+          const auto offset = offsets[slot];
+          if (offset < 0 || offset > offset_limit) {
+            return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+          }
+          if (size > offset_limit - offset) {
+            return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+          }
+        } else if (size < 0) {
+          return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+        }
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
+    const bool non_empty = data.length > 0;
     if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+      // For length 0, an empty offsets buffer is accepted (ARROW-544).

Review Comment:
   I don't think we want to extend this special-case to list views. This is (should be?) historical.
   Same for sizes below.



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -40,37 +40,46 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// List builder
+// VarLengthListLikeBuilder
 
 template <typename TYPE>
-class BaseListBuilder : public ArrayBuilder {
+class ARROW_EXPORT VarLengthListLikeBuilder : public ArrayBuilder {

Review Comment:
   I would keep the `Base` prefix to stress that it's not the final implementation class, e.g. `BaseVarLengthListBuilder`.



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,173 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > offset_limit");
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {

Review Comment:
   You can use `VisitSetBitRuns` which should make this massively simpler. For example:
   ```c++
   VisitSetBitRuns(validity, data.offset, data.length, [&](int64_t run_start, int64_t run_length) {
     for (int i = run_start; i < run_length; ++i) {
       const auto size = sizes[i];
       if (size > 0) {
         const auto offset = offsets[i];
         if (offset < 0 || offset > offset_limit) {
           return OutOfBoundsListViewOffset<offset_type>(i, offset_limit);
         }
         if (size > offset_limit - offset) {
           return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
         }
       } else if (size < 0) {
         return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
       }
     }
   });
   ```
   



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, input.offset, /*length=*/i + 1, [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; ++j) {
+          const auto offset = static_cast<int64_t>(offsets[input.offset + i + j]);
+          const auto size = sizes[input.offset + i + j];
+          if (size > 0) {
+            if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+              if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+                // 64-bit overflow detected. This is not possible on a valid list-view,
+                // but we saturate max_end to the maximum possible value to avoid
+                // undefined behavior.
+                max_end = kInt64Max;
+                return;
+              }
+            }
+            max_end = std::max(max_end, offset + size);
+          }
+        }
+      });
+  return max_end;
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByListView(const ArraySpan& input) {
+  DCHECK(is_list_view(*input.type));
+  if (input.length == 0 || input.GetNullCount() == input.length) {
+    return {0, 0};
+  }
+  const int64_t min_offset = MinViewOffset<offset_type>(input);
+  const int64_t max_end = MaxViewEnd<offset_type>(input);
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByList(const ArraySpan& input) {
+  DCHECK(is_var_length_list(*input.type));
+  if (input.length == 0) {
+    return {0, 0};
+  }
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const int64_t min_offset = offsets[input.offset];
+  const int64_t max_end = offsets[input.offset + input.length];
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename DestListViewType, typename SrcListType>
+Result<std::shared_ptr<ArrayData>> ListViewFromListImpl(
+    const std::shared_ptr<ArrayData>& list_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListType::offset_type,
+                   typename DestListViewType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename SrcListType::offset_type;
+  const auto& list_type = checked_cast<const SrcListType&>(*list_data->type);
+
+  // To re-use the validity and offsets buffers, a sizes buffer with enough
+  // padding on the beginning is allocated and filled with the sizes after
+  // list_data->offset.

Review Comment:
   You must initialize the initial padding area, otherwise you might leak private data accross IPC or the C Data Interface.



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(

Review Comment:
   Note there's a `ReverseSetBitRunReader` which can help apply the early exit inside the loop as suggested below for `MinViewOffset`.



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }

Review Comment:
   This prologue is a bit weird. The early exit when finding offset 0 could also apply inside the loop. 
   In other words, you can use `SetBitRunReader` so that you can `break` from inside your own loop.
   
   



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the

Review Comment:
   But `list_util_test.cc` showcases appending on the value builder before calling this...
   
   I think the API contract is ok as written, but then our tests shouldn't abuse the API?



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   What is the point of exposing this API on ListBuilder if `sizes` endures costly validation but is then entirely ignored? It doesn't seem to me like a very good idea, as builders are expected to be efficient.
   
   There is probably a limit to the amount of compatibility we can reasonably afford between ListBuilder and ListViewBuilder. And if we want some compatible vector-append, then I suggest to do the reverse: 
   expose a `AppendValues(const offset_type* offsets, int64_t length, const uint8_t* valid_bytes)` and let ListViewBuilder compute the sizes.
   



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.

Review Comment:
   Should update this comment which seems to apply to `MinViewOffset`, not `MaxViewEnd`.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(

Review Comment:
   I just rewrote everything 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// value builder.
+  ///
+  /// \pre if is_valid is false, list_length MUST be 0
+  /// \param is_valid Whether the new list slot is valid
+  /// \param list_length The number of elements in the list
+  Status Append(bool is_valid, int64_t list_length) {

Review Comment:
   The builders are mirroring the same sub-typing chain I created for the Arrays.
   
   ![image](https://github.com/apache/arrow/assets/207795/0603f862-550a-484d-8940-082122ff5181)
   



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/integration/json_internal.cc:
##########
@@ -1492,6 +1506,14 @@ class ArrayReader {
     return CreateList<T>(type_);
   }
 
+  Status Visit(const ListViewType& type) {
+    return Status::NotImplemented("list-view in JSON");
+  }
+
+  Status Visit(const LargeListViewType& type) {
+    return Status::NotImplemented("large list-view in JSON");
+  }

Review Comment:
   I implemented it for Go. I will add it to this PR before merging.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   @pitrou if you prefer that I rebase the PR and squash these `fixup!` commits, tell me. I'm leaving it like this to give you a chance to review the commits I added after your first review one by 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,171 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.

Review Comment:
   After I go through the impl, I make clear what does this mean. But the comment is a bit confusing, because this function has 3 arguments( specially offsets, sizes ). I guess we can make it 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -120,10 +135,14 @@ class ARROW_EXPORT ListArray : public BaseListArray<ListType> {
   /// This function does the bare minimum of validation of the offsets and
   /// input types, and will allocate a new offsets array if necessary (i.e. if
   /// the offsets contain any nulls). If the offsets do not have nulls, they
-  /// are assumed to be well-formed
+  /// are assumed to be well-formed.
   ///
-  /// Offsets of an Array's null bitmap can be present or an explicit
-  /// null_bitmap, but not both.
+  /// If a null_bitmap is not provided, the nulls will be inferred from the offsets' or
+  /// sizes' null bitmap. Only one of these two is allowed to have a null bitmap. But if a
+  /// null_bitmap is provided, the offsets array and the sizes array can't have nulls.

Review Comment:
   Ooops, it doesn't have `sizes` here, right?



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -113,33 +122,30 @@ Status ConcatenateOffsets(const BufferVector& buffers, MemoryPool* pool,
   values_ranges->resize(buffers.size());
 
   // allocate output buffer
-  int64_t out_length = 0;
-  for (const auto& buffer : buffers) {
-    out_length += buffer->size() / sizeof(Offset);
-  }
-  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer((out_length + 1) * sizeof(Offset), pool));
-  auto dst = reinterpret_cast<Offset*>((*out)->mutable_data());
+  const int64_t out_size_in_bytes = SumBufferSizesInBytes(buffers);
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(sizeof(Offset) + out_size_in_bytes, pool));

Review Comment:
   Just to make sure I understand this, actually this allocate one more element for ListView?



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -160,16 +166,69 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+void PutListViewOffsets(const Buffer& src, offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const BufferVector& buffers,

Review Comment:
   A quick stupid question: would the size here 1 greater than `sizes`?



-- 
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] felipecrv commented on pull request #35345: GH-35344: [C++][Format] Draft implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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

   Issue listing things this initial list-view implementation doesn't cover right away: https://github.com/apache/arrow/issues/37249


-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    if (offsets[i] == end_offset) {
+      end_offset += sizes[i];
+    } else {
+      non_null_fragments.push_back(
+          SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+      first_i = i;
+      end_offset = offsets[i] + sizes[i];
+    }
+  }
+  if (first_i >= 0) {
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+  }
+
+  // Final attempt to avoid invoking Concatenate().
+  if (non_null_fragments.size() == 1) {
+    return non_null_fragments[0];
+  } else if (non_null_fragments.size() == 0) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  return Concatenate(non_null_fragments, memory_pool);

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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -23,7 +23,7 @@
 #include "arrow/extension_type.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"
-#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_run_reader.h"

Review Comment:
   `validate.cc` didn't use any of the "counter" stuff. And after the spec simplifications that require null list-views to have valid dimensions, I don't even need the "reader" stuff anymore.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   I'm rewording the comments a bit to promise less regarding the validation.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/c/bridge_test.cc:
##########
@@ -3631,6 +3824,13 @@ TEST_F(TestArrayRoundtrip, List) {
   TestWithJSONSliced(fixed_size_list(int32(), 3), "[[4, 5, 6], null, [7, 8, null]]");
 }
 
+TEST_F(TestArrayRoundtrip, ListView) {
+  TestWithJSON(list_view(int32()), "[]");
+  TestWithJSON(list_view(int32()), "[[4, 5], [6, null], null]");

Review Comment:
   Alright, I just added a new test example in a fixup commit.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -379,6 +389,15 @@ class NullArrayFactory {
     enable_if_var_size_list<T, Status> Visit(const T& type) {
       // values array may be empty, but there must be at least one offset of 0
       RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * (length_ + 1)));
+      // XXX(felipec): reviewers, is this correct?
+      RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));
+      return Status::OK();
+    }
+
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T& type) {
+      RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * length_));
+      // XXX(felipec): reviewers, is this correct?

Review Comment:
   Looks right 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   I also wrapped the validation so it happens only in debug builds.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/type_traits.cc:
##########
@@ -67,21 +67,23 @@ int RequiredValueAlignmentForBuffer(Type::type type_id, int buffer_index) {
     case Type::BINARY:  // Offsets may be cast to int32_t*
     case Type::DATE32:
     case Type::TIME32:
-    case Type::LIST:  // Offsets may be cast to int32_t*, data is in child array
-    case Type::MAP:   // This is a list array
+    case Type::LIST:       // Offsets may be cast to int32_t*, data is in child array
+    case Type::LIST_VIEW:  // Same as LIST

Review Comment:
   No need to go into so much detail. Take this to mean that "aligment requirements of numeric buffers are the same between list and list-views".



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -280,6 +279,17 @@ class ArrayDataEndianSwapper {
     return Status::OK();
   }
 
+  Status Visit(const ListViewType& type) {
+    RETURN_NOT_OK(SwapOffsets<int32_t>(1));
+    RETURN_NOT_OK(SwapOffsets<int32_t>(2));
+    return Status::OK();
+  }
+  Status Visit(const LargeListViewType& type) {

Review Comment:
   `clang-format` doesn't put a line break here and I did it like this to match the two functions for `List` and `LargeList` right above 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,172 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.
+  ///
+  /// \param[in] offsets An array of int32 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int32 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the list-views in this array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> offsets() const;
+
+  /// \brief Return list-view sizes as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> sizes() const;
+
+ protected:
+  // This constructor defers SetData to a derived array class
+  ListViewArray() = default;
+
+  void SetData(const std::shared_ptr<ArrayData>& data);
+};
+
+/// \brief Concrete Array class for large list-view data (with 64-bit offsets
+/// and sizes)
+class ARROW_EXPORT LargeListViewArray : public BaseListViewArray<LargeListViewType> {
+ public:
+  explicit LargeListViewArray(std::shared_ptr<ArrayData> data);
+
+  LargeListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                     std::shared_ptr<Buffer> value_offsets,
+                     std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                     std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                     int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct LargeListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an LargeListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. TODO: describe the minimum validation

Review Comment:
   oops, here is a todo, should we keep this?



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,172 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.
+  ///
+  /// \param[in] offsets An array of int32 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int32 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the list-views in this array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> offsets() const;
+
+  /// \brief Return list-view sizes as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> sizes() const;
+
+ protected:
+  // This constructor defers SetData to a derived array class
+  ListViewArray() = default;
+
+  void SetData(const std::shared_ptr<ArrayData>& data);
+};
+
+/// \brief Concrete Array class for large list-view data (with 64-bit offsets
+/// and sizes)
+class ARROW_EXPORT LargeListViewArray : public BaseListViewArray<LargeListViewType> {
+ public:
+  explicit LargeListViewArray(std::shared_ptr<ArrayData> data);
+
+  LargeListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                     std::shared_ptr<Buffer> value_offsets,
+                     std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                     std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                     int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct LargeListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an LargeListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. TODO: describe the minimum validation
+  ///
+  /// TODO: review this

Review Comment:
   Still keep this?



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -97,10 +99,18 @@ Status ConcatenateBitmaps(const std::vector<Bitmap>& bitmaps, MemoryPool* pool,
   return Status::OK();
 }
 
+int64_t SumBufferSizes(const BufferVector& buffers) {

Review Comment:
   Would `SizeInBytes` better 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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -797,57 +811,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > ",
+                             offset_limit);
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    for (int64_t i = 0; i < data.length; ++i) {
+      const auto size = sizes[i];
+      if (size >= 0) {
+        const auto offset = offsets[i];
+        if (offset < 0 || offset > offset_limit) {
+          return OutOfBoundsListViewOffset<offset_type>(i, offset_limit);
+        }
+        if (size > offset_limit - offset) {
+          return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+        }
+      } else {
+        return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
-    if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+    const bool non_empty = data.length > 0;
+    if constexpr (is_list_view) {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return Status::Invalid("offsets buffer is null");
+      }
+      if (!IsBufferValid(2)) {
+        return Status::Invalid("sizes buffer is null");
+      }
+    } else {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return non_empty ? Status::Invalid("Non-empty array but offsets are null")
+                         : Status::OK();
       }
-      return Status::OK();
     }
 
-    // An empty list array can have 0 offsets
     const auto offsets_byte_size = data.buffers[1]->size();
     const auto required_offsets = ((data.length > 0) || (offsets_byte_size > 0))
-                                      ? data.length + data.offset + 1
+                                      ? data.length + data.offset + (is_list_view ? 0 : 1)
                                       : 0;
     if (offsets_byte_size / static_cast<int32_t>(sizeof(offset_type)) <
         required_offsets) {
       return Status::Invalid("Offsets buffer size (bytes): ", offsets_byte_size,
                              " isn't large enough for length: ", data.length,
                              " and offset: ", data.offset);
     }
+    if constexpr (is_list_view) {
+      const auto required_sizes = data.length + data.offset;
+      const auto sizes_bytes_size = data.buffers[2]->size();
+      if (sizes_bytes_size / static_cast<int32_t>(sizeof(offset_type)) < required_sizes) {
+        return Status::Invalid("Sizes buffer size (bytes): ", sizes_bytes_size,
+                               " isn't large enough for length: ", data.length,
+                               " and offset: ", data.offset);
+      }
+    }
 
     if (full_validation && required_offsets > 0) {
-      // Validate all offset values
-      const offset_type* offsets = data.GetValues<offset_type>(1);
-
-      auto prev_offset = offsets[0];
-      if (prev_offset < 0) {
-        return Status::Invalid(
-            "Offset invariant failure: array starts at negative offset ", prev_offset);
-      }
-      for (int64_t i = 1; i <= data.length; ++i) {
-        const auto current_offset = offsets[i];
-        if (current_offset < prev_offset) {
-          return Status::Invalid(
-              "Offset invariant failure: non-monotonic offset at slot ", i, ": ",
-              current_offset, " < ", prev_offset);
-        }
-        if (current_offset > offset_limit) {
-          return Status::Invalid("Offset invariant failure: offset for slot ", i,
-                                 " out of bounds: ", current_offset, " > ", offset_limit);
-        }
-        prev_offset = current_offset;
+      if constexpr (is_list_view) {
+        return FullyValidateOffsetsAndSizes<offset_type>(offset_limit);
+      } else {
+        return FullyValidateOffsets<offset_type>(offset_limit);
       }
     }
     return Status::OK();
   }
 
+ public:
+  template <typename TypeClass>
+  enable_if_list_view<TypeClass, Status> ValidateOffsetsAndSizes(const TypeClass& type,
+                                                                 int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }
+
+  template <typename TypeClass>
+  std::enable_if_t<is_var_length_list_type<TypeClass>::value ||
+                       is_base_binary_like(TypeClass::type_id),
+                   Status>
+  ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }

Review Comment:
   > But this class is just an implementation detail, it's not for outside use.
   
   But in a 1000-line file like this I find it beneficial to be explicit about the functions that can only be called within the current class.
   
   I will push a fixup with simplications.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -379,6 +389,15 @@ class NullArrayFactory {
     enable_if_var_size_list<T, Status> Visit(const T& type) {
       // values array may be empty, but there must be at least one offset of 0
       RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * (length_ + 1)));
+      // XXX(felipec): reviewers, is this correct?
+      RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));
+      return Status::OK();
+    }
+
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T& type) {
+      RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * length_));
+      // XXX(felipec): reviewers, is this correct?

Review Comment:
   I think it's correct and the above as well. @bkietz Perhaps you want to check this?



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,123 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
-  /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// This function should be called before appending elements to the
+  /// value builder. Elements appended to the value builder before this function
+  /// is called for the first time, will not be members of any list value.
+  ///
+  /// After this function is called, list_length elements SHOULD be appended to
+  /// the values builder. If this contract is violated, the behavior is defined by
+  /// the concrete builder implementation and SHOULD NOT be relied upon unless
+  /// the caller is specifically building a [Large]List or [Large]ListView array.
+  ///
+  /// For [Large]List arrays, the list slot length will be the number of elements
+  /// appended to the values builder before the next call to Append* or Finish. For
+  /// [Large]ListView arrays, the list slot length will be exactly list_length, but if
+  /// Append* is called before at least list_length elements are appended to the values
+  /// builder, the current list slot will share elements with the next list
+  /// slots or an invalid [Large]ListView array will be generated because there
+  /// aren't enough elements in the values builder to fill the list slots.
+  ///

Review Comment:
   ```suggestion
     ///
     /// If you're building a [Large]List and don't need to be compatible
     /// with [Large]ListView, then `BaseListBuilder::Append(bool is_valid)`
     /// is a simpler API.
     ///
   ```



##########
cpp/src/arrow/array/util.cc:
##########
@@ -379,6 +389,15 @@ class NullArrayFactory {
     enable_if_var_size_list<T, Status> Visit(const T& type) {
       // values array may be empty, but there must be at least one offset of 0
       RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * (length_ + 1)));
+      // XXX(felipec): reviewers, is this correct?
+      RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));
+      return Status::OK();
+    }
+
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T& type) {
+      RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * length_));
+      // XXX(felipec): reviewers, is this correct?
       RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));

Review Comment:
   Here as well, can pass 0 as a length.
   
   This will be exercised if there are nested list cases in the tests (such as list(list_view(...)), list(list(...)), etc.).



##########
cpp/src/arrow/c/bridge_test.cc:
##########
@@ -3631,6 +3824,13 @@ TEST_F(TestArrayRoundtrip, List) {
   TestWithJSONSliced(fixed_size_list(int32(), 3), "[[4, 5, 6], null, [7, 8, null]]");
 }
 
+TEST_F(TestArrayRoundtrip, ListView) {
+  TestWithJSON(list_view(int32()), "[]");
+  TestWithJSON(list_view(int32()), "[[4, 5], [6, null], null]");

Review Comment:
   Can we also check that non trivial list views (with overlapping or disjoint elements) roundtrip as well?



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +223,130 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  Status Append(bool is_valid = true) {
+    // The value_length parameter to BASE::Append(bool, int64_t) is ignored when
+    // building a list array, so we can pass 0 here.
+    return BASE::Append(is_valid, 0);
+  }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first length-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.

Review Comment:
   ```suggestion
       // on building a list that requires offsets to be non-decreasing.
       // CAUTION: the last size element (`sizes[length - 1]`) is not
       // validated and could be inconsistent with the offsets given in a
       // subsequent call to AppendValues.
   ```



##########
cpp/src/arrow/array/util.cc:
##########
@@ -379,6 +389,15 @@ class NullArrayFactory {
     enable_if_var_size_list<T, Status> Visit(const T& type) {
       // values array may be empty, but there must be at least one offset of 0
       RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * (length_ + 1)));
+      // XXX(felipec): reviewers, is this correct?
+      RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));

Review Comment:
   The list child array can be 0-length since all list elements will have length 0. This matches `NullArrayFactory::Visit`.
   ```suggestion
         RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), /*length=*/ 0)));
   ```



##########
cpp/src/arrow/array/array_list_test.cc:
##########
@@ -434,16 +622,18 @@ class TestListArray : public ::testing::Test {
 
   void TestBulkAppendInvalid() {
     std::vector<int16_t> values = {0, 1, 2, 3, 4, 5, 6};
-    std::vector<int> lengths = {3, 0, 4};
     std::vector<uint8_t> is_valid = {1, 0, 1};
     // Should be {0, 3, 3} given the is_valid array

Review Comment:
   Can we fix this comment?



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   I understand the concern, but this means that:
   1. the caller makes an effort to generate an array of sizes
   2. the callee validates that the array of sizes is "correct"
   3. the callee then proceeds to ignore the array of sizes
   
   (this even happens in optimized builds)
   
   I'll also note that `sizes[length - 1]` is not validated, so if I call in sequence:
   1. `Append(offsets = [0, 4, 5], sizes = [4, 1, 3], length = 3, valid_bytes = [1, 1, 1])`
   2. `Append(offsets = [6, 8], sizes = [2, 1], length = 2, valid_bytes = [1, 1])`
   
   ... then the third size (3) is incorrect yet silently accepted.
   
   Can we perhaps guard the validation with `#ifndef NDEBUG`?



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,23 +260,126 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT, bool HasNulls>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_offset = list_view_array.offset();
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if constexpr (HasNulls) {
+    if (list_view_array.null_count() == list_view_array.length()) {
+      return MakeEmptyArray(value_array->type(), memory_pool);
+    }
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  auto is_null_or_empty = [&](int64_t i) {
+    if constexpr (HasNulls) {
+      if (!bit_util::GetBit(validity, list_view_array_offset + i)) {
+        return true;
+      }
+    }
+    return sizes[i] == 0;
+  };
+
+  // Index of the first valid, non-empty list-view.
+  int64_t first_i = 0;
+  for (; first_i < list_view_array_length; first_i++) {
+    if (!is_null_or_empty(first_i)) {
+      break;
+    }
+  }
+  // If all list-views are empty, return an empty array.
+  if (first_i == list_view_array_length) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  std::vector<std::shared_ptr<Array>> slices;
+  {

Review Comment:
   This algorithm looks fine. It will be quite efficient if the list-view array is mostly contiguous and ascending, much less if the list view elements are all over the place :-)
   
   Would you mind adding a comment about the perf expectations here?
   
   (sidenote: perhaps we need a `Concatenate` API variant that takes a `vector<ArraySpan>`...)
   



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {

Review Comment:
   Ah, sorry, I had missed the early return. Scratch that :-(



##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }
+
+namespace internal {
+
+/// \brief Calculate the smallest continuous range of values used by the
+/// var-length list-like input (list, map and list-view types).
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return A pair of (offset, length) describing the range
+ARROW_EXPORT Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(
+    const ArraySpan& input);
+
+/// \brief Calculate the sum of the sizes of all valid lists or list-views
+///
+/// This is usally the same as the length of the RangeOfValuesUsed() range, but
+/// it can be:
+/// - Smaller: when the child array constains many values that are not
+/// referenced by the lists or list-views in the parent array
+/// - Greater: when the list-views share child array ranges
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return The sum of all list or list-view sizes
+ARROW_EXPORT Result<int64_t> SumOfLogicalListSizes(const ArraySpan& input);
+
+/// \brief Build a ListViewArray from a ListArray
+ARROW_EXPORT Result<std::shared_ptr<ListViewArray>> ListViewFromList(
+    const ListArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListViewArray from a LargeListArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListViewArray>> ListViewFromList(
+    const LargeListArray& source, MemoryPool* pool);
+
+/// \brief Build a ListArray from a ListViewArray
+ARROW_EXPORT Result<std::shared_ptr<ListArray>> ListFromListView(
+    const ListViewArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListArray from a LargeListViewArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListArray>> ListFromListView(
+    const LargeListViewArray& source, MemoryPool* pool);

Review Comment:
   I don't think it's a problem to use a Builder in `array_nested.cc`. It would be a problem to start including builder APIs from `array_nested.h`.



##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));
+    if (ARROW_PREDICT_TRUE(i != j)) {
+      // Swap validity bits
+      if (validity) {
+        const bool valid_i = bit_util::GetBit(validity, data.offset + i);
+        const bool valid_j = bit_util::GetBit(validity, data.offset + i);
+        if (valid_i != valid_j) {
+          bit_util::SetBitTo(validity, data.offset + i, valid_j);
+          bit_util::SetBitTo(validity, data.offset + j, valid_i);
+        }
+      }
+      // Swap offsets and sizes
+      std::swap(offsets[i], offsets[j]);
+      std::swap(sizes[i], sizes[j]);
+    }
+  }
+}
+
+/// \brief Generate the list-view offsets based on a random buffer of sizes.
+///
+/// The sizes buffer is an input of this function, but when force_empty_nulls is true,
+/// some values on the sizes buffer can be set to 0.
+///
+/// When sparsity is 0.0, the list-view spans are perfectly packed one after the
+/// other. If sparsity is greater than 0.0, the list-view spans are set apart
+/// from each other in proportion to the sparsity value and size of each
+/// list-view. A negative sparsity means each list-view shares a fraction of the
+/// values used by the previous list-view.
+///
+/// For instance, a sparsity of -1.0 means the values array will only need enough values
+/// for the largest list-view with all the other list-views spanning some of these same
+/// values.
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] mutable_sizes_array The array of sizes to use
+/// \param[in] force_empty_nulls Whether to force null list-view sizes to be 0
+/// \param[in] zero_undefined_offsets Whether to zero the offsets of list-views that have
+/// 0 set as the size
+/// \param[in] sparsity The sparsity of the generated list-view offsets
+/// \param[out] out_max_view_end The maximum value of the end of a list-view
+template <typename OffsetArrayType, typename offset_type>
+std::shared_ptr<Array> ViewOffsetsFromLengthsArray(
+    SeedType seed, OffsetArrayType& mutable_sizes_array, bool force_empty_nulls,
+    bool zero_undefined_offsets, double sparsity, int64_t* out_max_view_end,
+    int64_t alignment, MemoryPool* memory_pool) {
+  using TypeClass = typename OffsetArrayType::TypeClass;
+
+  auto* sizes = mutable_sizes_array.data()->template GetMutableValues<offset_type>(1);
+
+  BufferVector buffers{2};
+  buffers[0] = NULLPTR;  // sizes can have nulls, offsets don't have to
+  buffers[1] = *AllocateBuffer(sizeof(offset_type) * mutable_sizes_array.length(),
+                               alignment, memory_pool);
+  auto offsets = buffers[1]->mutable_data_as<offset_type>();
+
+  double offset_base = 0.0;
+  offset_type max_view_end = 0;
+  for (int64_t i = 0; i < mutable_sizes_array.length(); ++i) {
+    const auto offset = static_cast<offset_type>(std::llround(offset_base));
+    if (mutable_sizes_array.IsNull(i)) {
+      if (force_empty_nulls) {
+        sizes[i] = 0;
+      }
+      offsets[i] = zero_undefined_offsets ? 0 : offset;
+    } else {
+      if (sizes[i] == 0) {
+        offsets[i] = zero_undefined_offsets ? 0 : offset;
+      } else {
+        offsets[i] = offset;
+        DCHECK_LT(offset, std::numeric_limits<offset_type>::max() - sizes[i]);
+        offset_base = std::max(0.0, offset_base + (sparsity * sizes[i]));
+      }
+    }
+    max_view_end = std::max(max_view_end, offsets[i] + sizes[i]);
+  }
+  *out_max_view_end = max_view_end;
+
+  auto array_data =
+      ArrayData::Make(TypeTraits<TypeClass>::type_singleton(),
+                      mutable_sizes_array.length(), std::move(buffers), /*null_count=*/0);
+  return std::make_shared<OffsetArrayType>(std::move(array_data));
+}
+
+template <typename ArrayType, typename RAG>
+Result<std::shared_ptr<Array>> ArrayOfListView(RAG& self, const Field& field,

Review Comment:
   Could these at least use a similar `coverage` parameter? I don't think `sparsity` brings much to the table, and the fact that its description needs 8 lines of text probably means it won't get much use in practice.



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -797,57 +811,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > ",
+                             offset_limit);
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    for (int64_t i = 0; i < data.length; ++i) {
+      const auto size = sizes[i];
+      if (size >= 0) {
+        const auto offset = offsets[i];
+        if (offset < 0 || offset > offset_limit) {
+          return OutOfBoundsListViewOffset<offset_type>(i, offset_limit);
+        }
+        if (size > offset_limit - offset) {
+          return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+        }
+      } else {
+        return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
-    if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+    const bool non_empty = data.length > 0;
+    if constexpr (is_list_view) {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return Status::Invalid("offsets buffer is null");
+      }
+      if (!IsBufferValid(2)) {
+        return Status::Invalid("sizes buffer is null");
+      }
+    } else {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return non_empty ? Status::Invalid("Non-empty array but offsets are null")
+                         : Status::OK();
       }
-      return Status::OK();
     }
 
-    // An empty list array can have 0 offsets
     const auto offsets_byte_size = data.buffers[1]->size();
     const auto required_offsets = ((data.length > 0) || (offsets_byte_size > 0))
-                                      ? data.length + data.offset + 1
+                                      ? data.length + data.offset + (is_list_view ? 0 : 1)
                                       : 0;
     if (offsets_byte_size / static_cast<int32_t>(sizeof(offset_type)) <
         required_offsets) {
       return Status::Invalid("Offsets buffer size (bytes): ", offsets_byte_size,
                              " isn't large enough for length: ", data.length,
                              " and offset: ", data.offset);
     }
+    if constexpr (is_list_view) {
+      const auto required_sizes = data.length + data.offset;
+      const auto sizes_bytes_size = data.buffers[2]->size();
+      if (sizes_bytes_size / static_cast<int32_t>(sizeof(offset_type)) < required_sizes) {
+        return Status::Invalid("Sizes buffer size (bytes): ", sizes_bytes_size,
+                               " isn't large enough for length: ", data.length,
+                               " and offset: ", data.offset);
+      }
+    }
 
     if (full_validation && required_offsets > 0) {
-      // Validate all offset values
-      const offset_type* offsets = data.GetValues<offset_type>(1);
-
-      auto prev_offset = offsets[0];
-      if (prev_offset < 0) {
-        return Status::Invalid(
-            "Offset invariant failure: array starts at negative offset ", prev_offset);
-      }
-      for (int64_t i = 1; i <= data.length; ++i) {
-        const auto current_offset = offsets[i];
-        if (current_offset < prev_offset) {
-          return Status::Invalid(
-              "Offset invariant failure: non-monotonic offset at slot ", i, ": ",
-              current_offset, " < ", prev_offset);
-        }
-        if (current_offset > offset_limit) {
-          return Status::Invalid("Offset invariant failure: offset for slot ", i,
-                                 " out of bounds: ", current_offset, " > ", offset_limit);
-        }
-        prev_offset = current_offset;
+      if constexpr (is_list_view) {
+        return FullyValidateOffsetsAndSizes<offset_type>(offset_limit);
+      } else {
+        return FullyValidateOffsets<offset_type>(offset_limit);
       }
     }
     return Status::OK();
   }
 
+ public:
+  template <typename TypeClass>
+  enable_if_list_view<TypeClass, Status> ValidateOffsetsAndSizes(const TypeClass& type,
+                                                                 int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }
+
+  template <typename TypeClass>
+  std::enable_if_t<is_var_length_list_type<TypeClass>::value ||
+                       is_base_binary_like(TypeClass::type_id),
+                   Status>
+  ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }

Review Comment:
   But this class is just an implementation detail, it's not for outside use. 
   I don't entirely object to this but I found the indirection a bit tedious to follow when reading this 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint

Review Comment:
   True. They have to be densely packed. I will change the comment.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util_test.cc:
##########
@@ -0,0 +1,228 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array/builder_nested.h"
+#include "arrow/util/list_util.h"
+
+#include "arrow/testing/builder.h"
+#include "arrow/testing/gtest_util.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+using ListAndListViewTypes =
+    ::testing::Types<ListType, LargeListType, ListViewType, LargeListViewType>;
+
+template <typename T>
+class TestListUtils : public ::testing::Test {
+ public:
+  using TypeClass = T;
+  using offset_type = typename TypeClass::offset_type;
+  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
+
+  void SetUp() override {
+    value_type_ = int16();
+    type_ = std::make_shared<T>(value_type_);
+
+    std::unique_ptr<ArrayBuilder> tmp;
+    ASSERT_OK(MakeBuilder(pool_, type_, &tmp));
+    builder_.reset(checked_cast<BuilderType*>(tmp.release()));
+  }
+
+  void TestRangeOfValuesUsed() {
+    std::shared_ptr<ArrayData> result;
+
+    // Empty list-like array
+    ASSERT_OK(builder_->FinishInternal(&result));

Review Comment:
   `ArrayFromJSON` can't create list-views with overlapping ranges and other weird variations that are important for `RangeOfValuesUsed`.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   There seems to be some conflicts with main now :( you'll have to rebase and fix those.


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -160,16 +168,144 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+Status PutListViewOffsets(const ArrayData& input, offset_type* sizes, const Buffer& src,
+                          offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+//
+// The child arrays and the sizes buffer are used to ensure we can trust the offsets in
+// offset_buffers to be within the valid range.
+//
+// This function also mutates sizes so that null list-view entries have size 0.
+//
+// \param[in] in The child arrays
+// \param[in,out] sizes The concatenated sizes buffer
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const ArrayDataVector& in, offset_type* sizes,
+                                  const BufferVector& offset_buffers,
+                                  const std::vector<Range>& value_ranges,
+                                  MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  DCHECK_EQ(offset_buffers.size(), value_ranges.size());
+
+  // Allocate resulting offsets buffer and initialize it with zeros
+  const int64_t out_size_in_bytes = SumBufferSizesInBytes(offset_buffers);
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(out_size_in_bytes, pool));
+  memset((*out)->mutable_data(), 0, static_cast<size_t>((*out)->size()));
+
+  auto* out_offsets = (*out)->mutable_data_as<offset_type>();
+
+  int64_t num_child_values = 0;
+  int64_t elements_length = 0;
+  for (size_t i = 0; i < offset_buffers.size(); ++i) {
+    const auto displacement =
+        static_cast<offset_type>(num_child_values - value_ranges[i].offset);
+    RETURN_NOT_OK(PutListViewOffsets(*in[i], /*sizes=*/sizes + elements_length,
+                                     /*src=*/*offset_buffers[i], displacement,
+                                     /*dst=*/out_offsets + elements_length));
+    elements_length += offset_buffers[i]->size() / sizeof(offset_type);
+    num_child_values += value_ranges[i].length;
+    if (num_child_values > std::numeric_limits<offset_type>::max()) {
+      return Status::Invalid("offset overflow while concatenating arrays");
+    }
+  }
+  DCHECK_EQ(elements_length,
+            static_cast<int64_t>(out_size_in_bytes / sizeof(offset_type)));
+
+  return Status::OK();
+}
+
+template <typename offset_type>
+Status PutListViewOffsets(const ArrayData& input, offset_type* sizes, const Buffer& src,
+                          offset_type displacement, offset_type* dst) {
+  if (src.size() == 0) {
+    return Status::OK();
+  }
+  const auto& validity_buffer = input.buffers[0];
+  if (validity_buffer) {
+    // Ensure that it is safe to access all the bits in the validity bitmap of input.
+    RETURN_NOT_OK(internal::CheckSliceParams(/*size=*/8 * validity_buffer->size(),
+                                             input.offset, input.length, "buffer"));
+  }
+
+  const auto offsets = src.data_as<offset_type>();
+  DCHECK_EQ(static_cast<int64_t>(src.size() / sizeof(offset_type)), input.length);
+
+  auto visit_not_null = [&](int64_t position) {
+    if (sizes[position] > 0) {
+      // NOTE: Concatenate can be called during IPC reads to append delta
+      // dictionaries. Avoid UB on non-validated input by doing the addition in the
+      // unsigned domain. (the result can later be validated using
+      // Array::ValidateFull)
+      const auto displaced_offset = SafeSignedAdd(offsets[position], displacement);
+      // displaced_offset>=0 is guaranteed by RangeOfValuesUsed returning the
+      // smallest offset of valid and non-empty list-views.
+      DCHECK_GE(displaced_offset, 0);
+      dst[position] = displaced_offset;
+    } else {
+      // Do nothing to leave the dst[position] as 0.
+    }
+  };
+
+  const auto* validity = validity_buffer ? validity_buffer->data_as<uint8_t>() : nullptr;
+  internal::OptionalBitBlockCounter bit_counter(validity, input.offset, input.length);
+  int64_t position = 0;
+  while (position < input.length) {
+    internal::BitBlockCount block = bit_counter.NextBlock();
+    if (block.AllSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        if (sizes[position] > 0) {
+          // NOTE: Concatenate can be called during IPC reads to append delta
+          // dictionaries. Avoid UB on non-validated input by doing the addition in the
+          // unsigned domain. (the result can later be validated using
+          // Array::ValidateFull)
+          const auto displaced_offset = SafeSignedAdd(offsets[position], displacement);
+          // displaced_offset>=0 is guaranteed by RangeOfValuesUsed returning the
+          // smallest offset of valid and non-empty list-views.
+          DCHECK_GE(displaced_offset, 0);
+          dst[position] = displaced_offset;
+        } else {
+          // Do nothing to leave dst[position] as 0.
+        }

Review Comment:
   You're right. I extracted the function from below when I noticed the dup, but forgot to do the reverse-inlining above.
   
   Pushing 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -159,16 +166,67 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+void PutListViewOffsets(const Buffer& src, offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const BufferVector& buffers,
+                                  const std::vector<Range>& value_ranges,
+                                  MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  const int64_t out_size = SumBufferSizes(buffers);
+  if (out_size > std::numeric_limits<offset_type>::max()) {
+    return Status::Invalid("offset overflow while concatenating arrays");
+  }

Review Comment:
   Indeed. Fixing it in a dedicated commit now.
   
   I think this file would be less confusing if less math was done on buffer sizes (can't be trusted) and more reliance on the offset and length of arrays and array slices.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -113,33 +122,30 @@ Status ConcatenateOffsets(const BufferVector& buffers, MemoryPool* pool,
   values_ranges->resize(buffers.size());
 
   // allocate output buffer
-  int64_t out_length = 0;
-  for (const auto& buffer : buffers) {
-    out_length += buffer->size() / sizeof(Offset);
-  }
-  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer((out_length + 1) * sizeof(Offset), pool));
-  auto dst = reinterpret_cast<Offset*>((*out)->mutable_data());
+  const int64_t out_size_in_bytes = SumBufferSizesInBytes(buffers);
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(sizeof(Offset) + out_size_in_bytes, pool));

Review Comment:
   This is used only for lists. Not list-views and lists need one extra offset.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -853,6 +885,13 @@ class RepeatedArrayFactory {
     return builder.Finish(out);
   }
 
+  template <typename IntType>
+  Status CreateIntBuffer(IntType value, std::shared_ptr<Buffer>* out) {

Review Comment:
   Done. Pushing commit 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -181,7 +199,14 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   /// This function does the bare minimum of validation of the offsets and
   /// input types, and will allocate a new offsets array if necessary (i.e. if
   /// the offsets contain any nulls). If the offsets do not have nulls, they
-  /// are assumed to be well-formed
+  /// are assumed to be well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the
+  /// offsets's null bitmap. But if a null_bitmap is provided, the offsets array
+  /// can't have nulls.
+  ///
+  /// If a null_bitmap is provided, the offsets array can't be a slice (i.e. an
+  /// array with offset() > 0).

Review Comment:
   I amended the last commit and force-pushed.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -105,6 +103,23 @@ class BaseListArray : public Array {
   const offset_type* raw_value_offsets_ = NULLPTR;
 };
 
+// ----------------------------------------------------------------------
+// ListArray / LargeListArray
+
+template <typename TYPE>
+class BaseListArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_type() const { return this->var_length_list_like_type(); }
+
+  offset_type value_length(int64_t i) const final {

Review Comment:
   Yes, unlocking inlining of the call. And all the other complicated scenarios where the compiler is able to devirtualize.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/data.cc:
##########
@@ -233,9 +233,22 @@ BufferSpan OffsetsForScalar(uint8_t* scratch_space, offset_type value_size) {
   auto* offsets = reinterpret_cast<offset_type*>(scratch_space);
   offsets[0] = 0;
   offsets[1] = static_cast<offset_type>(value_size);
+  static_assert(2 * sizeof(offset_type) <= 16);
   return {scratch_space, sizeof(offset_type) * 2};
 }
 
+template <typename offset_type>
+std::pair<BufferSpan, BufferSpan> OffsetsAndSizesForScalar(uint8_t* scratch_space,
+                                                           offset_type value_size) {
+  auto* offsets = scratch_space;
+  auto* sizes = scratch_space + sizeof(offset_type);
+  reinterpret_cast<offset_type*>(offsets)[0] = 0;
+  reinterpret_cast<offset_type*>(sizes)[0] = value_size;

Review Comment:
   I'm going to replace many `reinterpret_casts` with `mutable_data_as` and `data_as` but this one can't because `scratch_space` is just a pointer, not a `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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/type.h:
##########
@@ -990,6 +990,71 @@ class ARROW_EXPORT LargeListType : public BaseListType {
   std::string ComputeFingerprint() const override;
 };
 
+/// \brief Type class for array of list views
+class ARROW_EXPORT ListViewType : public BaseListType {
+ public:
+  static constexpr Type::type type_id = Type::LIST_VIEW;
+  using offset_type = int32_t;
+
+  static constexpr const char* type_name() { return "list_view"; }
+
+  // ListView can contain any other logical value type
+  explicit ListViewType(const std::shared_ptr<DataType>& value_type)

Review Comment:
   I'm being consistent with `ListType` and `LargeListType` (and all the parametric types). I would like to fix these constructors and type factories systematically instead of diverging from the existing pattern on this 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,173 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > offset_limit");
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+              }
+              if (size > offset_limit - offset) {
+                return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+              }
+            } else if (size < 0) {
+              return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+            }
+          }
+        }
+      }
+    } else {
+      for (; slot < data.length; slot++) {
+        const auto size = sizes[slot];
+        if (size > 0) {
+          const auto offset = offsets[slot];
+          if (offset < 0 || offset > offset_limit) {
+            return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+          }
+          if (size > offset_limit - offset) {
+            return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+          }
+        } else if (size < 0) {
+          return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+        }
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
+    const bool non_empty = data.length > 0;
     if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+      // For length 0, an empty offsets buffer is accepted (ARROW-544).

Review Comment:
   I think it's natural that this issue will affect list views the same way it has affected lists.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   I'm a bit busy on these days, I'll take a careful round before this Sunday


-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }

Review Comment:
   I will rewrite this, but I only want to compare against 0 in the loop body if `offsets[i] < min_offset`, hence the need to compare `min_offset` against 0 before entering 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }

Review Comment:
   An advantage of skipping the check is that when `size[i]==0` we don't have to require the offset to be anything specific as long as the next list-view starts right where the previous non-empty list-view ended.
   
   Your comment made me realize one problem though: if the list-view arrays starts or ends with an empty list-view I could potentially use very random offsets to construct the slice at the end. The new implementation reduces the number of special cases, has an informal proof of correctness by induction, and more tests.



-- 
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] bkietz commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util_test.cc:
##########
@@ -0,0 +1,228 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array/builder_nested.h"
+#include "arrow/util/list_util.h"
+
+#include "arrow/testing/builder.h"
+#include "arrow/testing/gtest_util.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+using ListAndListViewTypes =
+    ::testing::Types<ListType, LargeListType, ListViewType, LargeListViewType>;
+
+template <typename T>
+class TestListUtils : public ::testing::Test {
+ public:
+  using TypeClass = T;
+  using offset_type = typename TypeClass::offset_type;
+  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
+
+  void SetUp() override {
+    value_type_ = int16();
+    type_ = std::make_shared<T>(value_type_);
+
+    std::unique_ptr<ArrayBuilder> tmp;
+    ASSERT_OK(MakeBuilder(pool_, type_, &tmp));
+    builder_.reset(checked_cast<BuilderType*>(tmp.release()));
+  }
+
+  void TestRangeOfValuesUsed() {
+    std::shared_ptr<ArrayData> result;
+
+    // Empty list-like array
+    ASSERT_OK(builder_->FinishInternal(&result));

Review Comment:
   Great, could you add comments explaining why these arrays are strange cases (and therefore require more manual construction)



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {

Review Comment:
   The first version looks like what I initially written, but it contains one extra boolean test
   
   `if (first_i == kUninitialized) {`
   
   This is exactly what I wanted to get rid of with the two loops. When I looked past the aversion to having multiple loops, I could read the code and informally prove its correctness by induction: first loop establishes the base case and the second loop builds on that. Final check at the end for cases with only nulls or empty list-views.
   
   Can I incorporate some of your style and do this?
   
   ```cpp
     constexpr int64_t kUninitialized = -1;
     int64_t first_i = kUninitialized;
     offset_type end_offset;
     int64_t i = 0;
     for (; i < list_view_array_length; i++) {
       if (is_null_or_empty(i)) continue;
   
       first_i = i;
       end_offset = offsets[i] + sizes[i];
       break;
     }
     i += 1;
     for (; i < list_view_array_length; i++) {
       if (is_null_or_empty(i)) continue;
   
       if (offsets[i] == end_offset) {
         end_offset += sizes[i];
         continue;
       }
       non_null_fragments.push_back(
           SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
       first_i = i;
       end_offset = offsets[i] + sizes[i];
     }
     if (first_i != kUninitialized) {
       non_null_fragments.push_back(
           SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
     }
   
   ```



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {

Review Comment:
   The first version looks like what I initially written, but it contains one extra boolean test
   
   `if (first_i == kUninitialized) {`
   
   This is exactly what I wanted to get rid of with the two loops. When I looked past the aversion to having multiple loops, I could read the code and informally prove its correctness by induction: first loop establishes the base case and the second loop builds on that. Final check at the end for cases with only nulls or empty list-views.
   
   Can incorporate some of your style and do this?
   
   ```cpp
     constexpr int64_t kUninitialized = -1;
     int64_t first_i = kUninitialized;
     offset_type end_offset;
     int64_t i = 0;
     for (; i < list_view_array_length; i++) {
       if (is_null_or_empty(i)) continue;
   
       first_i = i;
       end_offset = offsets[i] + sizes[i];
       break;
     }
     i += 1;
     for (; i < list_view_array_length; i++) {
       if (is_null_or_empty(i)) continue;
   
       if (offsets[i] == end_offset) {
         end_offset += sizes[i];
         continue;
       }
       non_null_fragments.push_back(
           SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
       first_i = i;
       end_offset = offsets[i] + sizes[i];
     }
     if (first_i != kUninitialized) {
       non_null_fragments.push_back(
           SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
     }
   
   ```



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -137,6 +137,78 @@ Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListArrayFromArray
   return std::make_shared<ArrayType>(std::move(data));
 }
 
+template <typename TYPE>
+Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListViewArrayFromArrays(
+    std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+    const Array& values, MemoryPool* pool, std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+    int64_t null_count = kUnknownNullCount) {
+  using offset_type = typename TYPE::offset_type;
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+  if (offsets.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List offsets must be ", OffsetArrowType::type_name());
+  }
+
+  if (sizes.length() != offsets.length() && sizes.length() != offsets.length() - 1) {
+    return Status::Invalid(
+        "List sizes must have the same length as offsets or one less than offsets");
+  }
+  if (sizes.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List sizes must be ", OffsetArrowType::type_name());
+  }
+
+  if (offsets.offset() != sizes.offset()) {
+    return Status::Invalid("List offsets and sizes must have the same offset");
+  }
+  const int64_t offset = sizes.offset();
+
+  if (null_bitmap) {
+    if (offsets.null_count() > 0 || sizes.null_count() > 0) {
+      return Status::Invalid(
+          "Ambiguous to specify both validity map and offsets or sizes with nulls");
+    }
+    if (offset != 0) {
+      return Status::Invalid(
+          "List offsets and sizes must not be slices if a validity map is specified");
+    }
+  } else {
+    if (offsets.null_count() > 0 && sizes.null_count() > 0) {
+      return Status::Invalid("Ambiguous to specify both offsets and sizes with nulls");
+    }
+  }
+
+  DCHECK(offsets.length() == sizes.length() || offsets.length() - 1 == sizes.length());
+
+  using OffsetArrayType = typename TypeTraits<OffsetArrowType>::ArrayType;
+  const auto& typed_offsets = checked_cast<const OffsetArrayType&>(offsets);
+  const auto& typed_sizes = checked_cast<const OffsetArrayType&>(sizes);
+
+  auto derived_validity_buffer = std::move(null_bitmap);
+  int64_t null_count_ = null_count;
+  if (offsets.null_count() > 0) {
+    derived_validity_buffer = offsets.null_bitmap();
+    null_count_ = offsets.null_count();
+    // We allow construction from an offsets array containing one extra value.
+    // If that is the case, we might need to discount one null from out_null_count.
+    if (offsets.length() - 1 == sizes.length() && !offsets.IsValid(sizes.length())) {
+      null_count_ -= 1;

Review Comment:
   An initial version of this code compared against `null_count` in the end. I will remove the extra variable 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,188 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };

Review Comment:
   Oh yeah. I can do this 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,173 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > offset_limit");
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+              }
+              if (size > offset_limit - offset) {
+                return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+              }
+            } else if (size < 0) {
+              return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+            }
+          }
+        }
+      }
+    } else {
+      for (; slot < data.length; slot++) {
+        const auto size = sizes[slot];
+        if (size > 0) {
+          const auto offset = offsets[slot];
+          if (offset < 0 || offset > offset_limit) {
+            return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+          }
+          if (size > offset_limit - offset) {
+            return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+          }
+        } else if (size < 0) {
+          return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+        }
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
+    const bool non_empty = data.length > 0;
     if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+      // For length 0, an empty offsets buffer is accepted (ARROW-544).

Review Comment:
   But I can start being more strict and wait for 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,62 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }
+
+namespace internal {
+
+/// \brief Calculate the smallest continuous range of values used by the
+/// var-length list-like input (list, map and list-view types).
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return A pair of (offset, length) describing the range
+ARROW_EXPORT Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(
+    const ArraySpan& input);
+
+/// \brief Builds a ListViewArray from a ListArray

Review Comment:
   4 cases fixed and added to a fixup commit.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, input.offset, /*length=*/i + 1, [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; ++j) {
+          const auto offset = static_cast<int64_t>(offsets[input.offset + i + j]);
+          const auto size = sizes[input.offset + i + j];
+          if (size > 0) {
+            if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+              if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+                // 64-bit overflow detected. This is not possible on a valid list-view,
+                // but we saturate max_end to the maximum possible value to avoid
+                // undefined behavior.
+                max_end = kInt64Max;
+                return;
+              }
+            }
+            max_end = std::max(max_end, offset + size);
+          }
+        }
+      });
+  return max_end;
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByListView(const ArraySpan& input) {
+  DCHECK(is_list_view(*input.type));
+  if (input.length == 0 || input.GetNullCount() == input.length) {
+    return {0, 0};
+  }
+  const int64_t min_offset = MinViewOffset<offset_type>(input);
+  const int64_t max_end = MaxViewEnd<offset_type>(input);
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByList(const ArraySpan& input) {
+  DCHECK(is_var_length_list(*input.type));
+  if (input.length == 0) {
+    return {0, 0};
+  }
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const int64_t min_offset = offsets[input.offset];
+  const int64_t max_end = offsets[input.offset + input.length];
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename DestListViewType, typename SrcListType>
+Result<std::shared_ptr<ArrayData>> ListViewFromListImpl(
+    const std::shared_ptr<ArrayData>& list_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListType::offset_type,
+                   typename DestListViewType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename SrcListType::offset_type;
+  const auto& list_type = checked_cast<const SrcListType&>(*list_data->type);
+
+  // To re-use the validity and offsets buffers, a sizes buffer with enough
+  // padding on the beginning is allocated and filled with the sizes after
+  // list_data->offset.
+  const int64_t buffer_length = list_data->offset + list_data->length;
+  ARROW_ASSIGN_OR_RAISE(auto sizes_buffer,
+                        AllocateBuffer(buffer_length * sizeof(offset_type), pool));
+  const auto* offsets = list_data->template GetValues<offset_type>(1, 0);
+  auto* sizes = reinterpret_cast<offset_type*>(sizes_buffer->mutable_data());
+  for (int64_t i = list_data->offset; i < buffer_length; i++) {
+    sizes[i] = offsets[i + 1] - offsets[i];
+  }
+  BufferVector buffers = {list_data->buffers[0], list_data->buffers[1],
+                          std::move(sizes_buffer)};
+
+  return ArrayData::Make(std::make_shared<DestListViewType>(list_type.value_type()),
+                         list_data->length, std::move(buffers),
+                         {list_data->child_data[0]}, list_data->null_count,
+                         list_data->offset);
+}
+
+template <typename DestListType, typename SrcListViewType>
+Result<std::shared_ptr<ArrayData>> ListFromListViewImpl(
+    const std::shared_ptr<ArrayData>& list_view_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListViewType::offset_type,
+                   typename DestListType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename DestListType::offset_type;
+  using ListBuilderType = typename TypeTraits<DestListType>::BuilderType;
+
+  const auto& list_view_type =
+      checked_cast<const SrcListViewType&>(*list_view_data->type);
+  const auto& value_type = list_view_type.value_type();
+  const auto list_type = std::make_shared<DestListType>(value_type);
+
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<ArrayBuilder> value_builder,
+                        MakeBuilder(value_type, pool));
+  auto list_builder = std::make_shared<ListBuilderType>(pool, value_builder, list_type);
+  RETURN_NOT_OK(list_builder->Reserve(list_view_data->length));
+
+  ArraySpan values{*list_view_data->child_data[0]};
+  const auto* in_validity_bitmap = list_view_data->GetValues<uint8_t>(0);
+  const auto* in_offsets = list_view_data->GetValues<offset_type>(1);
+  const auto* in_sizes = list_view_data->GetValues<offset_type>(2);
+  for (int64_t i = 0; i < list_view_data->length; ++i) {
+    const bool is_valid =
+        !in_validity_bitmap ||
+        bit_util::GetBit(in_validity_bitmap, list_view_data->offset + i);
+    const int64_t size = is_valid ? in_sizes[i] : 0;
+    RETURN_NOT_OK(list_builder->Append(is_valid, size));
+    RETURN_NOT_OK(value_builder->AppendArraySlice(values, in_offsets[i], size));
+  }
+  std::shared_ptr<ArrayData> list_array_data;
+  RETURN_NOT_OK(list_builder->FinishInternal(&list_array_data));
+  return list_array_data;
+}
+
+}  // namespace
+
+Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(const ArraySpan& input) {
+  switch (input.type->id()) {
+    case Type::LIST:
+      return RangeOfValuesUsedByList<ListType::offset_type>(input);
+    case Type::MAP:
+      return RangeOfValuesUsedByList<MapType::offset_type>(input);
+    case Type::LARGE_LIST:
+      return RangeOfValuesUsedByList<LargeListType::offset_type>(input);
+    case Type::LIST_VIEW:
+      return RangeOfValuesUsedByListView<ListViewType::offset_type>(input);
+    case Type::LARGE_LIST_VIEW:
+      return RangeOfValuesUsedByListView<LargeListViewType::offset_type>(input);
+    default:
+      break;
+  }
+  DCHECK(!is_var_length_list_like(*input.type));
+  return Status::Invalid("RangeOfValuesUsed: input is not a var-length list-like 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,173 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > offset_limit");
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {

Review Comment:
   Done in a fixup now. I don't know I didn't use the utility right away.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.

Review Comment:
   It's gone 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types

Review Comment:
   I will delete this paragraph as it doesn't make sense here. But I will clarify why the call to `BASE::Append` works.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   > What is the point of exposing this API on ListBuilder if sizes endures costly validation but is then entirely ignored? It doesn't seem to me like a very good idea, as builders are expected to be efficient.
   
   `ListBuilder` still exposes `AppendValues(offsets, length, valid_bytes)` with no loss of efficiency. `AppendValues` with `sizes` is overridden here so that we fulfill the generic contract of building var-length lists (requiring offsets + sizes).
   
   The expensive validation exists to catch people making mistakes of passing invalid `sizes` through the generic interface. They can gain more efficiency by passing `nullptr` on the sizes or using the `ListViewBuilder` directly. 
   
   > And if we want some compatible vector-append, then I suggest to do the reverse:
   expose a AppendValues(const offset_type* offsets, int64_t length, const uint8_t* valid_bytes) and let ListViewBuilder compute the sizes.
   
   Doing that has its problems as well: I can't delegate to a version that takes explicit `sizes` in list-view builders without allocating a temporary buffer. But I will follow this approach and have two specialized versions for list-views.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,23 +260,126 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT, bool HasNulls>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_offset = list_view_array.offset();
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if constexpr (HasNulls) {
+    if (list_view_array.null_count() == list_view_array.length()) {
+      return MakeEmptyArray(value_array->type(), memory_pool);
+    }
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  auto is_null_or_empty = [&](int64_t i) {
+    if constexpr (HasNulls) {
+      if (!bit_util::GetBit(validity, list_view_array_offset + i)) {
+        return true;
+      }
+    }
+    return sizes[i] == 0;
+  };
+
+  // Index of the first valid, non-empty list-view.
+  int64_t first_i = 0;
+  for (; first_i < list_view_array_length; first_i++) {
+    if (!is_null_or_empty(first_i)) {
+      break;
+    }
+  }
+  // If all list-views are empty, return an empty array.
+  if (first_i == list_view_array_length) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  std::vector<std::shared_ptr<Array>> slices;
+  {

Review Comment:
   > Would you mind adding a comment about the perf expectations here?
   
   Sure. Adding more text to the docstring.
   
   > (sidenote: perhaps we need a Concatenate API variant that takes a vector<ArraySpan>...)
   
   I once designed an API that had vectors of non-owning objects and it was a world of pain. We might come up with something better for Concatenate someday -- something that doesn't involve `std::vector` even.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }
+
+namespace internal {
+
+/// \brief Calculate the smallest continuous range of values used by the
+/// var-length list-like input (list, map and list-view types).
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return A pair of (offset, length) describing the range
+ARROW_EXPORT Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(
+    const ArraySpan& input);
+
+/// \brief Calculate the sum of the sizes of all valid lists or list-views
+///
+/// This is usally the same as the length of the RangeOfValuesUsed() range, but
+/// it can be:
+/// - Smaller: when the child array constains many values that are not
+/// referenced by the lists or list-views in the parent array
+/// - Greater: when the list-views share child array ranges
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return The sum of all list or list-view sizes
+ARROW_EXPORT Result<int64_t> SumOfLogicalListSizes(const ArraySpan& input);
+
+/// \brief Build a ListViewArray from a ListArray
+ARROW_EXPORT Result<std::shared_ptr<ListViewArray>> ListViewFromList(
+    const ListArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListViewArray from a LargeListArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListViewArray>> ListViewFromList(
+    const LargeListArray& source, MemoryPool* pool);
+
+/// \brief Build a ListArray from a ListViewArray
+ARROW_EXPORT Result<std::shared_ptr<ListArray>> ListFromListView(
+    const ListViewArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListArray from a LargeListViewArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListArray>> ListFromListView(
+    const LargeListViewArray& source, MemoryPool* pool);

Review Comment:
   Moved all the conversion functions to the array classes now. And the tests are moved from list_util_test to list_array_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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {
+    for (int64_t i = 0; i < input.length; i++) {
+      MINIMIZE_MIN_VIEW_OFFSET(i);
+    }
+  } else {
+    SetBitRunReader reader(validity, input.offset, input.length);
+    while (true) {
+      const auto run = reader.NextRun();
+      if (run.length == 0) {
+        break;
+      }
+      for (int64_t i = run.position; i < run.position + run.length; ++i) {
+        MINIMIZE_MIN_VIEW_OFFSET(i);
+      }
+    }
+  }
+  return min_offset;
+
+#undef MINIMIZE_MIN_VIEW_OFFSET
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  const auto values_length = input.child_data[0].length;
+
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Early-exit: 64-bit overflow detected. This is not possible on a valid list-view,
+  // but we return the maximum possible value to avoid undefined behavior.

Review Comment:
   This is a public API. I can't guarantee on what kind of data it will be 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   @felipecrv We'll want to update https://github.com/apache/arrow/blob/main/docs/source/status.rst in a followup 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   > There seems to be some conflicts with main now :( you'll have to rebase and fix those.
   
   Rebased and squashed the fixups 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   > Well I think build the List with size is ok. But can we add some comment for how to use that? Since that would be a bit tricky.
   
   Sure. Added more documentation.
   


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,349 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {
+    for (int64_t i = 0; i < input.length; i++) {
+      MINIMIZE_MIN_VIEW_OFFSET(i);
+    }
+  } else {
+    SetBitRunReader reader(validity, input.offset, input.length);
+    while (true) {
+      const auto run = reader.NextRun();
+      if (run.length == 0) {
+        break;
+      }
+      for (int64_t i = run.position; i < run.position + run.length; ++i) {
+        MINIMIZE_MIN_VIEW_OFFSET(i);
+      }
+    }
+  }
+  return min_offset.value_or(0);

Review Comment:
   If all the list-views are ~null or~ empty, the `min_offset` option will be empty and I return `0` which is safe because the `MaxViewEnd` will also be 0 in this case.
   
   There is an opportunity of optimization here that I hadn't noticed before: I can return the option, and if it's empty I don't have to call `MaxViewEnd`.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   To clarify, I didn't mean merging to the release branch, but to `main`.


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,171 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.

Review Comment:
   Yeah. Changing it now for both lists and list-views.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/data.cc:
##########
@@ -233,9 +233,22 @@ BufferSpan OffsetsForScalar(uint8_t* scratch_space, offset_type value_size) {
   auto* offsets = reinterpret_cast<offset_type*>(scratch_space);
   offsets[0] = 0;
   offsets[1] = static_cast<offset_type>(value_size);
+  static_assert(2 * sizeof(offset_type) <= 16);
   return {scratch_space, sizeof(offset_type) * 2};
 }
 
+template <typename offset_type>
+std::pair<BufferSpan, BufferSpan> OffsetsAndSizesForScalar(uint8_t* scratch_space,
+                                                           offset_type value_size) {
+  auto* offsets = scratch_space;
+  auto* sizes = scratch_space + sizeof(offset_type);
+  reinterpret_cast<offset_type*>(offsets)[0] = 0;
+  reinterpret_cast<offset_type*>(sizes)[0] = value_size;

Review Comment:
   could `mutable_data_as()` used here?



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -137,6 +137,77 @@ Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListArrayFromArray
   return std::make_shared<ArrayType>(std::move(data));
 }
 
+template <typename TYPE>
+Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListViewArrayFromArrays(
+    std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+    const Array& values, MemoryPool* pool, std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+    int64_t null_count = kUnknownNullCount) {
+  using offset_type = typename TYPE::offset_type;
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+  if (offsets.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List offsets must be ", OffsetArrowType::type_name());
+  }
+
+  if (sizes.length() != offsets.length() && sizes.length() != offsets.length() - 1) {
+    return Status::Invalid(
+        "List sizes must have the same length as offsets or one less than offsets");
+  }
+  if (sizes.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List sizes must be ", OffsetArrowType::type_name());
+  }
+
+  if (offsets.offset() != sizes.offset()) {
+    return Status::Invalid("List offsets and sizes must have the same offset");
+  }
+  const int64_t offset = sizes.offset();

Review Comment:
   Maybe I'm a bit stupid, but this offset would be easiliy confused with `offsets`... Can we have better way to distinct them or current impl is good enough?



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }

Review Comment:
   Why do we skip offset check when `sizes[i - 1] == 0`?
   
   ```
   offsets: 1 2 3 2 3
   sizes:    1 1 0 1 1
   ```
   
   does this introduce overlap and break (3)?



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -137,6 +137,77 @@ Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListArrayFromArray
   return std::make_shared<ArrayType>(std::move(data));
 }
 
+template <typename TYPE>
+Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListViewArrayFromArrays(
+    std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+    const Array& values, MemoryPool* pool, std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+    int64_t null_count = kUnknownNullCount) {
+  using offset_type = typename TYPE::offset_type;
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+  if (offsets.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List offsets must be ", OffsetArrowType::type_name());
+  }
+
+  if (sizes.length() != offsets.length() && sizes.length() != offsets.length() - 1) {
+    return Status::Invalid(
+        "List sizes must have the same length as offsets or one less than offsets");
+  }
+  if (sizes.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List sizes must be ", OffsetArrowType::type_name());
+  }
+
+  if (offsets.offset() != sizes.offset()) {
+    return Status::Invalid("List offsets and sizes must have the same offset");
+  }

Review Comment:
   Hmmm should we denote it in comment or `array_nested.h`?



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,171 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.

Review Comment:
   Or here you just want to mean `Offsets Array's`



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,171 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.

Review Comment:
   After I go through the impl, I make clear what does this mean. But the comment is a bit confusing, because this function has 3 arguments( specially offsets, sizes ). I guess we can make it mroe 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }

Review Comment:
   You're right. I'm pushing a commit with a rewrite of this entire function now + more tests.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -379,6 +389,15 @@ class NullArrayFactory {
     enable_if_var_size_list<T, Status> Visit(const T& type) {
       // values array may be empty, but there must be at least one offset of 0
       RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * (length_ + 1)));
+      // XXX(felipec): reviewers, is this correct?
+      RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));

Review Comment:
   Adding a commit manual (to ensure `clang-format`).



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));
+    if (ARROW_PREDICT_TRUE(i != j)) {
+      // Swap validity bits
+      if (validity) {
+        const bool valid_i = bit_util::GetBit(validity, data.offset + i);
+        const bool valid_j = bit_util::GetBit(validity, data.offset + i);
+        if (valid_i != valid_j) {
+          bit_util::SetBitTo(validity, data.offset + i, valid_j);
+          bit_util::SetBitTo(validity, data.offset + j, valid_i);
+        }
+      }
+      // Swap offsets and sizes
+      std::swap(offsets[i], offsets[j]);
+      std::swap(sizes[i], sizes[j]);
+    }
+  }
+}
+
+/// \brief Generate the list-view offsets based on a random buffer of sizes.
+///
+/// The sizes buffer is an input of this function, but when force_empty_nulls is true,
+/// some values on the sizes buffer can be set to 0.
+///
+/// When sparsity is 0.0, the list-view spans are perfectly packed one after the
+/// other. If sparsity is greater than 0.0, the list-view spans are set apart
+/// from each other in proportion to the sparsity value and size of each
+/// list-view. A negative sparsity means each list-view shares a fraction of the
+/// values used by the previous list-view.
+///
+/// For instance, a sparsity of -1.0 means the values array will only need enough values
+/// for the largest list-view with all the other list-views spanning some of these same
+/// values.
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] mutable_sizes_array The array of sizes to use
+/// \param[in] force_empty_nulls Whether to force null list-view sizes to be 0
+/// \param[in] zero_undefined_offsets Whether to zero the offsets of list-views that have
+/// 0 set as the size
+/// \param[in] sparsity The sparsity of the generated list-view offsets
+/// \param[out] out_max_view_end The maximum value of the end of a list-view
+template <typename OffsetArrayType, typename offset_type>
+std::shared_ptr<Array> ViewOffsetsFromLengthsArray(
+    SeedType seed, OffsetArrayType& mutable_sizes_array, bool force_empty_nulls,
+    bool zero_undefined_offsets, double sparsity, int64_t* out_max_view_end,
+    int64_t alignment, MemoryPool* memory_pool) {
+  using TypeClass = typename OffsetArrayType::TypeClass;
+
+  auto* sizes = mutable_sizes_array.data()->template GetMutableValues<offset_type>(1);
+
+  BufferVector buffers{2};
+  buffers[0] = NULLPTR;  // sizes can have nulls, offsets don't have to
+  buffers[1] = *AllocateBuffer(sizeof(offset_type) * mutable_sizes_array.length(),
+                               alignment, memory_pool);
+  auto offsets = buffers[1]->mutable_data_as<offset_type>();
+
+  double offset_base = 0.0;
+  offset_type max_view_end = 0;
+  for (int64_t i = 0; i < mutable_sizes_array.length(); ++i) {
+    const auto offset = static_cast<offset_type>(std::llround(offset_base));
+    if (mutable_sizes_array.IsNull(i)) {
+      if (force_empty_nulls) {
+        sizes[i] = 0;
+      }
+      offsets[i] = zero_undefined_offsets ? 0 : offset;
+    } else {
+      if (sizes[i] == 0) {
+        offsets[i] = zero_undefined_offsets ? 0 : offset;
+      } else {
+        offsets[i] = offset;
+        DCHECK_LT(offset, std::numeric_limits<offset_type>::max() - sizes[i]);
+        offset_base = std::max(0.0, offset_base + (sparsity * sizes[i]));
+      }
+    }
+    max_view_end = std::max(max_view_end, offsets[i] + sizes[i]);
+  }
+  *out_max_view_end = max_view_end;
+
+  auto array_data =
+      ArrayData::Make(TypeTraits<TypeClass>::type_singleton(),
+                      mutable_sizes_array.length(), std::move(buffers), /*null_count=*/0);
+  return std::make_shared<OffsetArrayType>(std::move(array_data));
+}
+
+template <typename ArrayType, typename RAG>
+Result<std::shared_ptr<Array>> ArrayOfListView(RAG& self, const Field& field,

Review Comment:
   I was planning to use the sparsity concept in benchmarks as the tighter packing of the data could affect performance, but I will remove it for now and stash the code somewhere.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -69,33 +99,117 @@ class ConcatenateTest : public ::testing::Test {
     return slices;
   }
 
+  std::shared_ptr<Buffer> ValidityBitmap(int64_t size, double null_probability) {
+    return rag_.NullBitmap(size, null_probability, kDefaultBufferAlignment,
+                           default_memory_pool());
+  }
+
   template <typename PrimitiveType>
-  std::shared_ptr<Array> GeneratePrimitive(int64_t size, double null_probability) {
+  std::shared_ptr<Array> PrimitiveArray(int64_t size, double null_probability) {
     if (std::is_same<PrimitiveType, BooleanType>::value) {
-      return rng_.Boolean(size, 0.5, null_probability);
+      return rag_.Boolean(size, 0.5, null_probability);
     }
-    return rng_.Numeric<PrimitiveType, uint8_t>(size, 0, 127, null_probability);
+    return rag_.Numeric<PrimitiveType, uint8_t>(size, 0, 127, null_probability);
+  }
+
+  std::shared_ptr<Array> StringArray(int64_t size, double null_probability) {
+    return rag_.String(size, /*min_length =*/0, /*max_length =*/15, null_probability);
+  }
+
+  std::shared_ptr<Array> LargeStringArray(int64_t size, double null_probability) {
+    return rag_.LargeString(size, /*min_length =*/0, /*max_length =*/15,
+                            null_probability);
+  }
+
+  std::shared_ptr<Array> StringViewArray(int64_t size, double null_probability) {
+    return rag_.StringView(size, /*min_length =*/0, /*max_length =*/40, null_probability,
+                           /*max_buffer_length=*/200);
+  }
+
+  std::shared_ptr<Array> ArrayOf(std::shared_ptr<DataType> type, int64_t size,
+                                 double null_probability) {
+    return rag_.ArrayOf(std::move(type), size, null_probability);
+  }
+
+  // TODO(GH-38656): Use the random array generators from testing/random.h here
+
+  template <typename ListType,
+            typename ListArrayType = typename TypeTraits<ListType>::ArrayType>
+  Result<std::shared_ptr<ListArrayType>> ListArray(int32_t length,
+                                                   double null_probability) {
+    using offset_type = typename ListType::offset_type;
+    using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+    auto values_size = length * 4;
+    auto values = PrimitiveArray<Int8Type>(values_size, null_probability);
+    auto offsets_vector = Offsets<offset_type>(values_size, length);
+    // Ensure first and last offsets encompass the whole values array
+    offsets_vector.front() = 0;

Review Comment:
   @pitrou I also don't do this for list-views anymore.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));

Review Comment:
   I got it from cppreference and it seems to be The Right Way :tm: to change the sampling range instead of creating a new distribution every 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -187,33 +189,89 @@ TEST_F(ConcatenateTest, FixedSizeListType) {
   });
 }
 
-TEST_F(ConcatenateTest, ListType) {
-  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+template <typename ListType>
+struct ListConcatenationChecker {
+  using offset_type = typename ListType::offset_type;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+  using ListArrayType = typename TypeTraits<ListType>::ArrayType;
+
+  template <typename Self>
+  static void Check(Self& self, int32_t size, double null_probability,
+                    std::shared_ptr<Array>* out) {
     auto values_size = size * 4;
-    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
-    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    auto values =
+        self.template GeneratePrimitive<Int8Type>(values_size, null_probability);
+    auto offsets_vector = self.template Offsets<offset_type>(values_size, size);
     // Ensure first and last offsets encompass the whole values array
     offsets_vector.front() = 0;
-    offsets_vector.back() = static_cast<int32_t>(values_size);
+    offsets_vector.back() = static_cast<offset_type>(values_size);
     std::shared_ptr<Array> offsets;
-    ArrayFromVector<Int32Type>(offsets_vector, &offsets);
-    ASSERT_OK_AND_ASSIGN(*out, ListArray::FromArrays(*offsets, *values));
+    ArrayFromVector<OffsetArrowType>(offsets_vector, &offsets);
+    ASSERT_OK_AND_ASSIGN(*out, ListArrayType::FromArrays(*offsets, *values));
     ASSERT_OK((**out).ValidateFull());
+  }
+};
+
+TEST_F(ConcatenateTest, ListType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    ListConcatenationChecker<ListType>::Check(*this, size, null_probability, out);
   });
 }
 
 TEST_F(ConcatenateTest, LargeListType) {
   Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
-    auto values_size = size * 4;
-    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
-    auto offsets_vector = this->Offsets<int64_t>(values_size, size);
-    // Ensure first and last offsets encompass the whole values array
-    offsets_vector.front() = 0;
-    offsets_vector.back() = static_cast<int64_t>(values_size);
+    ListConcatenationChecker<LargeListType>::Check(*this, size, null_probability, out);
+  });
+}
+
+template <typename ListViewType>
+struct ListViewConcatenationChecker {
+  using offset_type = typename ListViewType::offset_type;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+  using ListViewArrayType = typename TypeTraits<ListViewType>::ArrayType;
+
+  template <typename Self>
+  static void Check(Self& self, int32_t num_list_views, double null_probability,
+                    std::shared_ptr<Array>* out) {
+    auto values_size = 4 * num_list_views;
+    auto values =
+        self.template GeneratePrimitive<Int8Type>(values_size, null_probability);
+
     std::shared_ptr<Array> offsets;
-    ArrayFromVector<Int64Type>(offsets_vector, &offsets);
-    ASSERT_OK_AND_ASSIGN(*out, LargeListArray::FromArrays(*offsets, *values));
+    auto offsets_vector = self.template Offsets<offset_type>(values_size, num_list_views);
+    offsets_vector.front() = 0;

Review Comment:
   Instead of improving these generators I propose we use `testing/random.h`. I created an issue for this now -> https://github.com/apache/arrow/issues/38656
   
   But I would prefer tackling this problem in another PR. It's bigger than list-view: it would benefit the list concatenation tests 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -797,57 +811,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > ",
+                             offset_limit);
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    for (int64_t i = 0; i < data.length; ++i) {
+      const auto size = sizes[i];
+      if (size >= 0) {
+        const auto offset = offsets[i];
+        if (offset < 0 || offset > offset_limit) {
+          return OutOfBoundsListViewOffset<offset_type>(i, offset_limit);
+        }
+        if (size > offset_limit - offset) {
+          return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+        }
+      } else {
+        return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
-    if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+    const bool non_empty = data.length > 0;
+    if constexpr (is_list_view) {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).

Review Comment:
   Sure. Forgot to remove it when I changed the 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -383,6 +393,12 @@ class NullArrayFactory {
       return Status::OK();
     }
 
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T&) {
+      buffer_length_ = length_ * sizeof(typename T::offset_type);

Review Comment:
   @bkietz rewrote this whole inner class in his PR didn't he? I hope I can merge this before his PR. :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] bkietz commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {

Review Comment:
   How do these seem to you:
   ```c++
     constexpr int64_t kUninitialized = -1;
     int64_t first_i = kUninitialized;
     offset_type end_offset;
     for (int64_t i = 0; i < list_view_array_length; i++) {
       // skip null and empty views
       if (validity && !bit_util::GetBit(validity, i)) continue;
       if (sizes[i] == 0) continue;
   
       if (first_i == kUninitialized) {
         first_i = i;
         end_offset = offsets[i] + sizes[i];
         break;
       }
   
       if (offsets[i] == end_offset) {
         // this view is contiguous with the current fragment; just fold it in
         end_offset += sizes[i];
         continue;
       }
   
       // slice out the current fragment, then set the current fragment to this view
       non_null_fragments.push_back(
           SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
       first_i = i;
       end_offset = offsets[i] + sizes[i];
     }
   ```
   
   ```c++
     int64_t first_i = -1;
     offset_type end_offset = -1;
     int64_t i = 0;
     auto skip_null_or_empty = [&] {
       for (; i < list_view_array_length; i++) {
         if (validity && !bit_util::GetBit(validity, i)) continue;
         if (sizes[i] == 0) continue;
         break;
       }
     };
   
     skip_null_or_empty();
     end_offset = offsets[i] + sizes[i];
     first_i = i++;
   
     while (i < list_view_array_length) {
       skip_null_or_empty();
   
       if (offsets[i] == end_offset) {
         // this view is contiguous with the current fragment; just fold it in
         end_offset += sizes[i++];
         continue;
       }
   
       // slice out the current fragment, then set the current fragment to this view
       non_null_fragments.push_back(
           SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
       end_offset = offsets[i] + sizes[i];
       first_i = i++;
     }
   ```



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// value builder.
+  ///
+  /// \pre if is_valid is false, list_length MUST be 0
+  /// \param is_valid Whether the new list slot is valid
+  /// \param list_length The number of elements in the list
+  Status Append(bool is_valid, int64_t list_length) {

Review Comment:
   I want the list-view builder to allow users to control the sizes they push into the sizes buffer.
   
   Example: I can append logical values `[4, 5, 6]`, `[4, 5]`, and `[4]` by
   
   ```
   Append(true, 3)
   Append(true, 2)
   Append(true, 1)
   child->Append([4, 5, 6])
   ```
   
   List of suffixes is also possible with careful alternations of child mutations and size appends.



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// value builder.
+  ///
+  /// \pre if is_valid is false, list_length MUST be 0
+  /// \param is_valid Whether the new list slot is valid
+  /// \param list_length The number of elements in the list
+  Status Append(bool is_valid, int64_t list_length) {

Review Comment:
   I want the list-view builder to allow users to control the sizes they push into the sizes buffer.
   
   Example: I can append logical values `[4, 5, 6]`, `[4, 5]`, and `[4]` by
   
   ```
   Append(true, 3)
   Append(true, 2)
   Append(true, 1)
   child->Append([4, 5, 6])
   ```
   
   List of suffixes is also possible with careful alternations of child mutations and offset+size appends.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -40,37 +40,46 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// List builder
+// VarLengthListLikeBuilder
 
 template <typename TYPE>
-class BaseListBuilder : public ArrayBuilder {
+class ARROW_EXPORT VarLengthListLikeBuilder : public ArrayBuilder {

Review Comment:
   ```
   Base+VarLengthListLikeBuilder
   
   or 
   
   Base+VarLengthListBuilder
   ```
   
   ?



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, input.offset, /*length=*/i + 1, [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; ++j) {
+          const auto offset = static_cast<int64_t>(offsets[input.offset + i + j]);
+          const auto size = sizes[input.offset + i + j];
+          if (size > 0) {
+            if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+              if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+                // 64-bit overflow detected. This is not possible on a valid list-view,
+                // but we saturate max_end to the maximum possible value to avoid
+                // undefined behavior.
+                max_end = kInt64Max;
+                return;
+              }
+            }
+            max_end = std::max(max_end, offset + size);
+          }
+        }
+      });
+  return max_end;
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByListView(const ArraySpan& input) {
+  DCHECK(is_list_view(*input.type));
+  if (input.length == 0 || input.GetNullCount() == input.length) {
+    return {0, 0};
+  }
+  const int64_t min_offset = MinViewOffset<offset_type>(input);
+  const int64_t max_end = MaxViewEnd<offset_type>(input);
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByList(const ArraySpan& input) {
+  DCHECK(is_var_length_list(*input.type));
+  if (input.length == 0) {
+    return {0, 0};
+  }
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const int64_t min_offset = offsets[input.offset];
+  const int64_t max_end = offsets[input.offset + input.length];
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename DestListViewType, typename SrcListType>
+Result<std::shared_ptr<ArrayData>> ListViewFromListImpl(
+    const std::shared_ptr<ArrayData>& list_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListType::offset_type,
+                   typename DestListViewType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename SrcListType::offset_type;
+  const auto& list_type = checked_cast<const SrcListType&>(*list_data->type);
+
+  // To re-use the validity and offsets buffers, a sizes buffer with enough
+  // padding on the beginning is allocated and filled with the sizes after
+  // list_data->offset.

Review Comment:
   Adding a `memset` 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/integration/json_internal.cc:
##########
@@ -1492,6 +1506,14 @@ class ArrayReader {
     return CreateList<T>(type_);
   }
 
+  Status Visit(const ListViewType& type) {
+    return Status::NotImplemented("list-view in JSON");
+  }
+
+  Status Visit(const LargeListViewType& type) {
+    return Status::NotImplemented("large list-view in JSON");
+  }

Review Comment:
   I implemented it for Go. For C++, should I have it in this PR or a separate PR is fine?



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -40,37 +40,46 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// List builder
+// VarLengthListLikeBuilder
 
 template <typename TYPE>
-class BaseListBuilder : public ArrayBuilder {
+class ARROW_EXPORT VarLengthListLikeBuilder : public ArrayBuilder {

Review Comment:
   ```
   Base+VarLengthListLikeBuilder
   
   or 
   
   Base+VarLengthListBuilder
   ```
   ?
   
   I like the -Like-



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, /*offset=*/input.offset + i + 1, /*length=*/input.length - i - 1,
+      [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; j++) {
+          const auto offset = offsets[input.offset + i + j];
+          if (ARROW_PREDICT_FALSE(offset < min_offset)) {
+            if (sizes[input.offset + i + j] > 0) {
+              min_offset = offset;
+            }
+          }
+        }
+      });
+  return min_offset;
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : NULLPTR;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+  const auto IsNull = [validity](int64_t i) -> bool {
+    return validity && !arrow::bit_util::GetBit(validity, i);
+  };
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset zero, so we check that first and potentially early-return a 0.
+  int64_t i = input.length - 1;  // safe because input.length() > 0
+  while (i != 0 && (IsNull(i) || sizes[input.offset + i] == 0)) {
+    i -= 1;
+  }
+  const auto offset = static_cast<int64_t>(offsets[input.offset + i]);
+  const auto size = sizes[input.offset + i];
+  if (i == 0) {
+    return (IsNull(i) || sizes[input.offset + i] == 0) ? 0 : offset + size;
+  }
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+    if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+      // Early-exit: 64-bit overflow detected. This is not possible on a
+      // valid list-view, but we return the maximum possible value to
+      // avoid undefined behavior.
+      return kInt64Max;
+    }
+  }
+  int64_t max_end =
+      static_cast<int64_t>(offsets[input.offset + i]) + sizes[input.offset + i];
+  if (max_end == input.child_data[0].length) {
+    // Early-exit: maximum possible view-end found already.
+    return max_end;
+  }
+
+  // Slow path: scan the buffers entirely.
+  arrow::internal::VisitSetBitRunsVoid(
+      validity, input.offset, /*length=*/i + 1, [&](int64_t i, int64_t run_length) {
+        for (int64_t j = 0; j < run_length; ++j) {
+          const auto offset = static_cast<int64_t>(offsets[input.offset + i + j]);
+          const auto size = sizes[input.offset + i + j];
+          if (size > 0) {
+            if constexpr (sizeof(offset_type) == sizeof(int64_t)) {
+              if (ARROW_PREDICT_FALSE(offset > kInt64Max - size)) {
+                // 64-bit overflow detected. This is not possible on a valid list-view,
+                // but we saturate max_end to the maximum possible value to avoid
+                // undefined behavior.
+                max_end = kInt64Max;
+                return;
+              }
+            }
+            max_end = std::max(max_end, offset + size);
+          }
+        }
+      });
+  return max_end;
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByListView(const ArraySpan& input) {
+  DCHECK(is_list_view(*input.type));
+  if (input.length == 0 || input.GetNullCount() == input.length) {
+    return {0, 0};
+  }
+  const int64_t min_offset = MinViewOffset<offset_type>(input);
+  const int64_t max_end = MaxViewEnd<offset_type>(input);
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename offset_type>
+std::pair<int64_t, int64_t> RangeOfValuesUsedByList(const ArraySpan& input) {
+  DCHECK(is_var_length_list(*input.type));
+  if (input.length == 0) {
+    return {0, 0};
+  }
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const int64_t min_offset = offsets[input.offset];
+  const int64_t max_end = offsets[input.offset + input.length];
+  return {min_offset, max_end - min_offset};
+}
+
+template <typename DestListViewType, typename SrcListType>
+Result<std::shared_ptr<ArrayData>> ListViewFromListImpl(
+    const std::shared_ptr<ArrayData>& list_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListType::offset_type,
+                   typename DestListViewType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename SrcListType::offset_type;
+  const auto& list_type = checked_cast<const SrcListType&>(*list_data->type);
+
+  // To re-use the validity and offsets buffers, a sizes buffer with enough
+  // padding on the beginning is allocated and filled with the sizes after
+  // list_data->offset.
+  const int64_t buffer_length = list_data->offset + list_data->length;
+  ARROW_ASSIGN_OR_RAISE(auto sizes_buffer,
+                        AllocateBuffer(buffer_length * sizeof(offset_type), pool));
+  const auto* offsets = list_data->template GetValues<offset_type>(1, 0);
+  auto* sizes = reinterpret_cast<offset_type*>(sizes_buffer->mutable_data());
+  for (int64_t i = list_data->offset; i < buffer_length; i++) {
+    sizes[i] = offsets[i + 1] - offsets[i];
+  }
+  BufferVector buffers = {list_data->buffers[0], list_data->buffers[1],
+                          std::move(sizes_buffer)};
+
+  return ArrayData::Make(std::make_shared<DestListViewType>(list_type.value_type()),
+                         list_data->length, std::move(buffers),
+                         {list_data->child_data[0]}, list_data->null_count,
+                         list_data->offset);
+}
+
+template <typename DestListType, typename SrcListViewType>
+Result<std::shared_ptr<ArrayData>> ListFromListViewImpl(
+    const std::shared_ptr<ArrayData>& list_view_data, MemoryPool* pool) {
+  static_assert(
+      std::is_same<typename SrcListViewType::offset_type,
+                   typename DestListType::offset_type>::value,
+      "Offset types between list type and list-view type are expected to match");
+  using offset_type = typename DestListType::offset_type;
+  using ListBuilderType = typename TypeTraits<DestListType>::BuilderType;
+
+  const auto& list_view_type =
+      checked_cast<const SrcListViewType&>(*list_view_data->type);
+  const auto& value_type = list_view_type.value_type();
+  const auto list_type = std::make_shared<DestListType>(value_type);
+
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<ArrayBuilder> value_builder,
+                        MakeBuilder(value_type, pool));
+  auto list_builder = std::make_shared<ListBuilderType>(pool, value_builder, list_type);
+  RETURN_NOT_OK(list_builder->Reserve(list_view_data->length));

Review Comment:
   I created a new utility called `SumOfLogicalListSizes` to use here. Pushing 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] bkietz commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,179 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -602,8 +691,11 @@ class ConcatenateImpl {
 }  // namespace
 
 Result<std::shared_ptr<Array>> Concatenate(const ArrayVector& arrays, MemoryPool* pool) {
-  if (arrays.size() == 0) {
-    return Status::Invalid("Must pass at least one array");
+  switch (arrays.size()) {

Review Comment:
   :ok_hand: 



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -248,6 +305,38 @@ class ConcatenateImpl {
     return ConcatenateImpl(child_data, pool_).Concatenate(&out_->child_data[0]);
   }
 
+  template <typename T>
+  enable_if_list_view<T, Status> Visit(const T& type) {
+    using offset_type = typename T::offset_type;
+    out_->buffers.resize(3);
+    out_->child_data.resize(1);
+
+    // Calculate the ranges of values that each list-view array uses
+    std::vector<Range> value_ranges;
+    value_ranges.reserve(in_.size());
+    for (const auto& input : in_) {
+      ArraySpan input_span(*input);
+      Range range;
+      ARROW_ASSIGN_OR_RAISE(std::tie(range.offset, range.length),
+                            list_util::internal::RangeOfValuesUsed(input_span));
+      value_ranges.push_back(range);

Review Comment:
   (Not actually a suggestion for this PR, just speculation about future helpers) what would you think of adding a macro for `ASSIGN_OR_RAISE` with structured binding?
   ```suggestion
         ARROW_ASSIGN_OR_RAISE_LIST((offset, length),
                                    list_util::internal::RangeOfValuesUsed(input_span));
         value_ranges.push_back({offset, 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   I've create an issue about parquet. https://github.com/apache/arrow/issues/38849


-- 
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] felipecrv commented on pull request #35345: GH-35344: [C++] Draft an implementation of the LIST_VIEW array format

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

   @bkietz 


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +231,172 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types.
+  ///
+  /// Offsets of an Array's null bitmap can be present or an explicit
+  /// null_bitmap, but not both.
+  ///
+  /// \param[in] offsets An array of int32 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int32 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the list-views in this array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> offsets() const;
+
+  /// \brief Return list-view sizes as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> sizes() const;
+
+ protected:
+  // This constructor defers SetData to a derived array class
+  ListViewArray() = default;
+
+  void SetData(const std::shared_ptr<ArrayData>& data);
+};
+
+/// \brief Concrete Array class for large list-view data (with 64-bit offsets
+/// and sizes)
+class ARROW_EXPORT LargeListViewArray : public BaseListViewArray<LargeListViewType> {
+ public:
+  explicit LargeListViewArray(std::shared_ptr<ArrayData> data);
+
+  LargeListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                     std::shared_ptr<Buffer> value_offsets,
+                     std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                     std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                     int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct LargeListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an LargeListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. TODO: describe the minimum validation

Review Comment:
   I could swear I had removed 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -97,10 +99,18 @@ Status ConcatenateBitmaps(const std::vector<Bitmap>& bitmaps, MemoryPool* pool,
   return Status::OK();
 }
 
+int64_t SumBufferSizes(const BufferVector& buffers) {

Review Comment:
   Yes. Renaming.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,349 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {
+    for (int64_t i = 0; i < input.length; i++) {
+      MINIMIZE_MIN_VIEW_OFFSET(i);
+    }
+  } else {
+    SetBitRunReader reader(validity, input.offset, input.length);
+    while (true) {
+      const auto run = reader.NextRun();
+      if (run.length == 0) {
+        break;
+      }
+      for (int64_t i = run.position; i < run.position + run.length; ++i) {
+        MINIMIZE_MIN_VIEW_OFFSET(i);
+      }
+    }
+  }
+  return min_offset.value_or(0);

Review Comment:
   Hmmm in this case(.i.e `min_offset == 0`) what does it means for return 0?



##########
cpp/src/arrow/array/util.cc:
##########
@@ -280,6 +279,17 @@ class ArrayDataEndianSwapper {
     return Status::OK();
   }
 
+  Status Visit(const ListViewType& type) {
+    RETURN_NOT_OK(SwapOffsets<int32_t>(1));
+    RETURN_NOT_OK(SwapOffsets<int32_t>(2));
+    return Status::OK();
+  }
+  Status Visit(const LargeListViewType& type) {

Review Comment:
   format?



##########
cpp/src/arrow/type_traits.cc:
##########
@@ -67,21 +67,23 @@ int RequiredValueAlignmentForBuffer(Type::type type_id, int buffer_index) {
     case Type::BINARY:  // Offsets may be cast to int32_t*
     case Type::DATE32:
     case Type::TIME32:
-    case Type::LIST:  // Offsets may be cast to int32_t*, data is in child array
-    case Type::MAP:   // This is a list array
+    case Type::LIST:       // Offsets may be cast to int32_t*, data is in child array
+    case Type::LIST_VIEW:  // Same as LIST

Review Comment:
   Offsets and sizes?



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/c/bridge.cc:
##########
@@ -444,6 +444,10 @@ struct SchemaExporter {
 
   Status Visit(const LargeListType& type) { return SetFormat("+L"); }
 
+  Status Visit(const ListViewType& type) { return SetFormat("+lv"); }
+
+  Status Visit(const LargeListViewType& type) { return SetFormat("+Lv"); }

Review Comment:
   Fixing up the commit changing the bridge 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/builder.cc:
##########
@@ -219,6 +219,20 @@ struct MakeBuilderImpl {
     return Status::OK();
   }
 
+  Status Visit(const ListViewType& list_view_type) {
+    std::shared_ptr<DataType> value_type = list_view_type.value_type();
+    ARROW_ASSIGN_OR_RAISE(auto value_builder, ChildBuilder(value_type));
+    out.reset(new ListViewBuilder(pool, std::move(value_builder), std::move(type)));
+    return Status::OK();
+  }
+
+  Status Visit(const LargeListViewType& large_list_view_type) {
+    std::shared_ptr<DataType> value_type = large_list_view_type.value_type();
+    ARROW_ASSIGN_OR_RAISE(auto value_builder, ChildBuilder(value_type));
+    out.reset(new LargeListViewBuilder(pool, std::move(value_builder), type));

Review Comment:
   In a fixup.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -203,6 +203,54 @@ TEST_F(ConcatenateTest, LargeListType) {
   });
 }
 
+TEST_F(ConcatenateTest, ListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    auto values_size = size * 4;
+    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
+
+    std::shared_ptr<Array> offsets;
+    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    offsets_vector.front() = 0;
+    offsets_vector.back() = values_size;

Review Comment:
   Unnecessary indeed. 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -40,37 +40,46 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// List builder
+// VarLengthListLikeBuilder
 
 template <typename TYPE>
-class BaseListBuilder : public ArrayBuilder {
+class ARROW_EXPORT VarLengthListLikeBuilder : public ArrayBuilder {

Review Comment:
   ```
   Base+VarLengthListLikeArray
   
   or 
   
   Base+VarLengthListArray
   ```
   
   ?



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/testing/random.h:
##########
@@ -438,6 +438,23 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
                               int64_t alignment = kDefaultBufferAlignment,
                               MemoryPool* memory_pool = default_memory_pool());
 
+  /// \brief Generate a random ListViewArray
+  ///
+  /// \param[in] values The underlying values array
+  /// \param[in] size The size of the generated list array
+  /// \param[in] null_probability the probability of a list value being null
+  /// \param[in] force_empty_nulls if true, null list entries must have 0 length

Review Comment:
   In a fixup 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/ipc/json_simple.cc:
##########
@@ -555,8 +563,9 @@ class ListConverter final : public ConcreteConverter<ListConverter<TYPE>> {
     if (json_obj.IsNull()) {
       return this->AppendNull();
     }
-    RETURN_NOT_OK(builder_->Append());
     // Extend the child converter with this JSON array
+    ARROW_ASSIGN_OR_RAISE(auto size, this->SizeOfJSONArray(json_obj));
+    RETURN_NOT_OK(builder_->Append(true, size));

Review Comment:
   Do you have a name suggestion for "Add a valid or invalid list of a given size"? Note that, per spec, nulls lists can have non-zero sizes in which case filler values are added to the child 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_base.cc:
##########
@@ -319,10 +324,30 @@ struct DerefConstIterator {
   pointer operator->() const { return &(**it); }
 };
 
+/// If A and B are equivalent types, a builder of type A can receive
+/// scalar values of type B and a builder of type B can receive
+/// scalar values of type A.
+///
+/// \param a Type A.
+/// \param b Type B.
+bool AreScalarTypesEquivalent(const DataType& a, const DataType& b) {
+  if (a.Equals(b)) {
+    return true;
+  }
+  if ((a.id() == Type::LIST && b.id() == Type::LIST_VIEW) ||
+      (a.id() == Type::LIST_VIEW && b.id() == Type::LIST) ||
+      (a.id() == Type::LARGE_LIST && b.id() == Type::LARGE_LIST_VIEW) ||
+      (a.id() == Type::LARGE_LIST_VIEW && b.id() == Type::LARGE_LIST)) {
+    return checked_cast<const BaseListType&>(a).value_type()->Equals(
+        *checked_cast<const BaseListType&>(b).value_type());
+  }
+  return false;
+}
+
 }  // namespace
 
 Status ArrayBuilder::AppendScalar(const Scalar& scalar, int64_t n_repeats) {
-  if (!scalar.type->Equals(type())) {
+  if (!AreScalarTypesEquivalent(*scalar.type, *type())) {

Review Comment:
   This simplified work I was doing in compute kernels tremendously. I think that if since we have different physical representation for the same logical types we should start thinking about these conversions. Perhaps not in the ad-hoc way I've done here but with a more robust and general type-checking algorithm.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/builder.cc:
##########
@@ -219,6 +219,20 @@ struct MakeBuilderImpl {
     return Status::OK();
   }
 
+  Status Visit(const ListViewType& list_view_type) {
+    std::shared_ptr<DataType> value_type = list_view_type.value_type();
+    ARROW_ASSIGN_OR_RAISE(auto value_builder, ChildBuilder(value_type));
+    out.reset(new ListViewBuilder(pool, std::move(value_builder), type));

Review Comment:
   Done in a fixup commit.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +241,183 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. The offset and length of the offsets and sizes arrays must
+  /// match and that will be checked, but their contents will be assumed to be
+  /// well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the
+  /// offsets's null bitmap. But if a null_bitmap is provided, the offsets array
+  /// can't have nulls.
+  ///
+  /// If a null_bitmap is provided, the offsets array can't be a slice (i.e. an
+  /// array with offset() > 0).
+  ///
+  /// \param[in] offsets An array of int32 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int32 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the list-views in this array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> offsets() const;
+
+  /// \brief Return list-view sizes as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> sizes() const;
+
+ protected:
+  // This constructor defers SetData to a derived array class
+  ListViewArray() = default;
+
+  void SetData(const std::shared_ptr<ArrayData>& data);
+};
+
+/// \brief Concrete Array class for large list-view data (with 64-bit offsets
+/// and sizes)
+class ARROW_EXPORT LargeListViewArray : public BaseListViewArray<LargeListViewType> {
+ public:
+  explicit LargeListViewArray(std::shared_ptr<ArrayData> data);
+
+  LargeListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                     std::shared_ptr<Buffer> value_offsets,
+                     std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                     std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                     int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct LargeListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an LargeListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. The offset and length of the offsets and sizes arrays must
+  /// match and that will be checked, but their contents will be assumed to be
+  /// well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the offsets' or
+  /// sizes' null bitmap. Only one of these two is allowed to have a null bitmap. But if a
+  /// null_bitmap is provided, the offsets array and the sizes array can't have nulls.
+  ///
+  /// And when a null_bitmap is provided, neither the offsets or sizes array can be a
+  /// slice (i.e. an array with offset() > 0).
+  ///
+  /// \param[in] offsets An array of int64 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int64 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<LargeListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<LargeListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the large list-views in this
+  /// array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int64Array

Review Comment:
   Pushing a commit fixing 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/integration/json_internal.cc:
##########
@@ -1492,6 +1506,14 @@ class ArrayReader {
     return CreateList<T>(type_);
   }
 
+  Status Visit(const ListViewType& type) {
+    return Status::NotImplemented("list-view in JSON");
+  }
+
+  Status Visit(const LargeListViewType& type) {
+    return Status::NotImplemented("large list-view in JSON");
+  }

Review Comment:
   Or should I do it in another PR since this one is quite overwhelming to 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));
+    if (ARROW_PREDICT_TRUE(i != j)) {
+      // Swap validity bits
+      if (validity) {
+        const bool valid_i = bit_util::GetBit(validity, data.offset + i);
+        const bool valid_j = bit_util::GetBit(validity, data.offset + i);
+        if (valid_i != valid_j) {
+          bit_util::SetBitTo(validity, data.offset + i, valid_j);
+          bit_util::SetBitTo(validity, data.offset + j, valid_i);
+        }
+      }
+      // Swap offsets and sizes
+      std::swap(offsets[i], offsets[j]);
+      std::swap(sizes[i], sizes[j]);
+    }
+  }
+}
+
+/// \brief Generate the list-view offsets based on a random buffer of sizes.
+///
+/// The sizes buffer is an input of this function, but when force_empty_nulls is true,
+/// some values on the sizes buffer can be set to 0.
+///
+/// When sparsity is 0.0, the list-view spans are perfectly packed one after the
+/// other. If sparsity is greater than 0.0, the list-view spans are set apart
+/// from each other in proportion to the sparsity value and size of each
+/// list-view. A negative sparsity means each list-view shares a fraction of the
+/// values used by the previous list-view.
+///
+/// For instance, a sparsity of -1.0 means the values array will only need enough values
+/// for the largest list-view with all the other list-views spanning some of these same
+/// values.
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] mutable_sizes_array The array of sizes to use
+/// \param[in] force_empty_nulls Whether to force null list-view sizes to be 0
+/// \param[in] zero_undefined_offsets Whether to zero the offsets of list-views that have
+/// 0 set as the size
+/// \param[in] sparsity The sparsity of the generated list-view offsets
+/// \param[out] out_max_view_end The maximum value of the end of a list-view
+template <typename OffsetArrayType, typename offset_type>
+std::shared_ptr<Array> ViewOffsetsFromLengthsArray(
+    SeedType seed, OffsetArrayType& mutable_sizes_array, bool force_empty_nulls,
+    bool zero_undefined_offsets, double sparsity, int64_t* out_max_view_end,
+    int64_t alignment, MemoryPool* memory_pool) {
+  using TypeClass = typename OffsetArrayType::TypeClass;
+
+  auto* sizes = mutable_sizes_array.data()->template GetMutableValues<offset_type>(1);
+
+  BufferVector buffers{2};
+  buffers[0] = NULLPTR;  // sizes can have nulls, offsets don't have to
+  buffers[1] = *AllocateBuffer(sizeof(offset_type) * mutable_sizes_array.length(),
+                               alignment, memory_pool);
+  auto offsets = buffers[1]->mutable_data_as<offset_type>();
+
+  double offset_base = 0.0;
+  offset_type max_view_end = 0;
+  for (int64_t i = 0; i < mutable_sizes_array.length(); ++i) {
+    const auto offset = static_cast<offset_type>(std::llround(offset_base));
+    if (mutable_sizes_array.IsNull(i)) {
+      if (force_empty_nulls) {
+        sizes[i] = 0;
+      }
+      offsets[i] = zero_undefined_offsets ? 0 : offset;
+    } else {
+      if (sizes[i] == 0) {
+        offsets[i] = zero_undefined_offsets ? 0 : offset;
+      } else {
+        offsets[i] = offset;
+        DCHECK_LT(offset, std::numeric_limits<offset_type>::max() - sizes[i]);
+        offset_base = std::max(0.0, offset_base + (sparsity * sizes[i]));
+      }
+    }
+    max_view_end = std::max(max_view_end, offsets[i] + sizes[i]);
+  }
+  *out_max_view_end = max_view_end;
+
+  auto array_data =
+      ArrayData::Make(TypeTraits<TypeClass>::type_singleton(),
+                      mutable_sizes_array.length(), std::move(buffers), /*null_count=*/0);
+  return std::make_shared<OffsetArrayType>(std::move(array_data));
+}
+
+template <typename ArrayType, typename RAG>
+Result<std::shared_ptr<Array>> ArrayOfListView(RAG& self, const Field& field,

Review Comment:
   https://github.com/apache/arrow/issues/38748



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/c/bridge_test.cc:
##########
@@ -3631,6 +3824,13 @@ TEST_F(TestArrayRoundtrip, List) {
   TestWithJSONSliced(fixed_size_list(int32(), 3), "[[4, 5, 6], null, [7, 8, null]]");
 }
 
+TEST_F(TestArrayRoundtrip, ListView) {
+  TestWithJSON(list_view(int32()), "[]");
+  TestWithJSON(list_view(int32()), "[[4, 5], [6, null], null]");

Review Comment:
   Well, the import side must compute the buffer length still, so not entirely trivial for list views?



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_list_test.cc:
##########
@@ -434,16 +622,18 @@ class TestListArray : public ::testing::Test {
 
   void TestBulkAppendInvalid() {
     std::vector<int16_t> values = {0, 1, 2, 3, 4, 5, 6};
-    std::vector<int> lengths = {3, 0, 4};
     std::vector<uint8_t> is_valid = {1, 0, 1};
     // Should be {0, 3, 3} given the is_valid array

Review Comment:
   +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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   I'm rewording the comments a bit to promise less about the validation.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -379,6 +389,15 @@ class NullArrayFactory {
     enable_if_var_size_list<T, Status> Visit(const T& type) {
       // values array may be empty, but there must be at least one offset of 0
       RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * (length_ + 1)));
+      // XXX(felipec): reviewers, is this correct?
+      RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));
+      return Status::OK();
+    }
+
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T& type) {
+      RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * length_));
+      // XXX(felipec): reviewers, is this correct?
       RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));

Review Comment:
   Fixup commit coming 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/c/bridge_test.cc:
##########
@@ -3631,6 +3824,13 @@ TEST_F(TestArrayRoundtrip, List) {
   TestWithJSONSliced(fixed_size_list(int32(), 3), "[[4, 5, 6], null, [7, 8, null]]");
 }
 
+TEST_F(TestArrayRoundtrip, ListView) {
+  TestWithJSON(list_view(int32()), "[]");
+  TestWithJSON(list_view(int32()), "[[4, 5], [6, null], null]");

Review Comment:
   Isn't that an overkill given that round-tripping is all about passing the buffers around correctly without any consideration about their contents?



-- 
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 #35345: GH-35344: [C++] Draft an implementation of the LIST_VIEW array format

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #35345:
URL: https://github.com/apache/arrow/pull/35345#issuecomment-1523502392

   :warning: GitHub issue #35344 **has been automatically assigned in GitHub** to PR creator.


-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util_test.cc:
##########
@@ -0,0 +1,228 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array/builder_nested.h"
+#include "arrow/util/list_util.h"
+
+#include "arrow/testing/builder.h"
+#include "arrow/testing/gtest_util.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+using ListAndListViewTypes =
+    ::testing::Types<ListType, LargeListType, ListViewType, LargeListViewType>;
+
+template <typename T>
+class TestListUtils : public ::testing::Test {
+ public:
+  using TypeClass = T;
+  using offset_type = typename TypeClass::offset_type;
+  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
+
+  void SetUp() override {
+    value_type_ = int16();
+    type_ = std::make_shared<T>(value_type_);
+
+    std::unique_ptr<ArrayBuilder> tmp;
+    ASSERT_OK(MakeBuilder(pool_, type_, &tmp));
+    builder_.reset(checked_cast<BuilderType*>(tmp.release()));
+  }
+
+  void TestRangeOfValuesUsed() {
+    std::shared_ptr<ArrayData> result;
+
+    // Empty list-like array
+    ASSERT_OK(builder_->FinishInternal(&result));

Review Comment:
   I also think (I'm not sure) I added these before I was able to make the JSON conversions work.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_base.cc:
##########
@@ -319,10 +324,30 @@ struct DerefConstIterator {
   pointer operator->() const { return &(**it); }
 };
 
+/// If A and B are equivalent types, a builder of type A can receive
+/// scalar values of type B and a builder of type B can receive
+/// scalar values of type A.
+///
+/// \param a Type A.
+/// \param b Type B.
+bool AreScalarTypesEquivalent(const DataType& a, const DataType& b) {
+  if (a.Equals(b)) {
+    return true;
+  }
+  if ((a.id() == Type::LIST && b.id() == Type::LIST_VIEW) ||
+      (a.id() == Type::LIST_VIEW && b.id() == Type::LIST) ||
+      (a.id() == Type::LARGE_LIST && b.id() == Type::LARGE_LIST_VIEW) ||
+      (a.id() == Type::LARGE_LIST_VIEW && b.id() == Type::LARGE_LIST)) {
+    return checked_cast<const BaseListType&>(a).value_type()->Equals(
+        *checked_cast<const BaseListType&>(b).value_type());
+  }
+  return false;
+}
+
 }  // namespace
 
 Status ArrayBuilder::AppendScalar(const Scalar& scalar, int64_t n_repeats) {
-  if (!scalar.type->Equals(type())) {
+  if (!AreScalarTypesEquivalent(*scalar.type, *type())) {

Review Comment:
   I will remove this and report what fails. If nothing fails, I will simply remove it, otherwise I will report here what happens.



-- 
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] bkietz commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_base.cc:
##########
@@ -319,10 +324,30 @@ struct DerefConstIterator {
   pointer operator->() const { return &(**it); }
 };
 
+/// If A and B are equivalent types, a builder of type A can receive
+/// scalar values of type B and a builder of type B can receive
+/// scalar values of type A.
+///
+/// \param a Type A.
+/// \param b Type B.
+bool AreScalarTypesEquivalent(const DataType& a, const DataType& b) {
+  if (a.Equals(b)) {
+    return true;
+  }
+  if ((a.id() == Type::LIST && b.id() == Type::LIST_VIEW) ||
+      (a.id() == Type::LIST_VIEW && b.id() == Type::LIST) ||
+      (a.id() == Type::LARGE_LIST && b.id() == Type::LARGE_LIST_VIEW) ||
+      (a.id() == Type::LARGE_LIST_VIEW && b.id() == Type::LARGE_LIST)) {
+    return checked_cast<const BaseListType&>(a).value_type()->Equals(
+        *checked_cast<const BaseListType&>(b).value_type());
+  }
+  return false;
+}
+
 }  // namespace
 
 Status ArrayBuilder::AppendScalar(const Scalar& scalar, int64_t n_repeats) {
-  if (!scalar.type->Equals(type())) {
+  if (!AreScalarTypesEquivalent(*scalar.type, *type())) {

Review Comment:
   I'm not opposed to the conversion, just to making it a property of the builder. I think if things do break then it'd be possible to recover functionality by extracting the conversion to `template <typename S> Result<S> EquivalentCast(const Scalar&);` (or maybe being more formal and adding `Scalar::View`) and calling that before AppendScalar



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -120,10 +135,14 @@ class ARROW_EXPORT ListArray : public BaseListArray<ListType> {
   /// This function does the bare minimum of validation of the offsets and
   /// input types, and will allocate a new offsets array if necessary (i.e. if
   /// the offsets contain any nulls). If the offsets do not have nulls, they
-  /// are assumed to be well-formed
+  /// are assumed to be well-formed.
   ///
-  /// Offsets of an Array's null bitmap can be present or an explicit
-  /// null_bitmap, but not both.
+  /// If a null_bitmap is not provided, the nulls will be inferred from the offsets' or
+  /// sizes' null bitmap. Only one of these two is allowed to have a null bitmap. But if a
+  /// null_bitmap is provided, the offsets array and the sizes array can't have nulls.

Review Comment:
   Copy and pasta mistake. Fixing 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));
+    if (ARROW_PREDICT_TRUE(i != j)) {
+      // Swap validity bits
+      if (validity) {
+        const bool valid_i = bit_util::GetBit(validity, data.offset + i);
+        const bool valid_j = bit_util::GetBit(validity, data.offset + i);
+        if (valid_i != valid_j) {
+          bit_util::SetBitTo(validity, data.offset + i, valid_j);
+          bit_util::SetBitTo(validity, data.offset + j, valid_i);
+        }
+      }
+      // Swap offsets and sizes
+      std::swap(offsets[i], offsets[j]);
+      std::swap(sizes[i], sizes[j]);
+    }
+  }
+}
+
+/// \brief Generate the list-view offsets based on a random buffer of sizes.
+///
+/// The sizes buffer is an input of this function, but when force_empty_nulls is true,
+/// some values on the sizes buffer can be set to 0.
+///
+/// When sparsity is 0.0, the list-view spans are perfectly packed one after the
+/// other. If sparsity is greater than 0.0, the list-view spans are set apart
+/// from each other in proportion to the sparsity value and size of each
+/// list-view. A negative sparsity means each list-view shares a fraction of the
+/// values used by the previous list-view.
+///
+/// For instance, a sparsity of -1.0 means the values array will only need enough values
+/// for the largest list-view with all the other list-views spanning some of these same
+/// values.
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] mutable_sizes_array The array of sizes to use
+/// \param[in] force_empty_nulls Whether to force null list-view sizes to be 0
+/// \param[in] zero_undefined_offsets Whether to zero the offsets of list-views that have
+/// 0 set as the size
+/// \param[in] sparsity The sparsity of the generated list-view offsets
+/// \param[out] out_max_view_end The maximum value of the end of a list-view
+template <typename OffsetArrayType, typename offset_type>
+std::shared_ptr<Array> ViewOffsetsFromLengthsArray(
+    SeedType seed, OffsetArrayType& mutable_sizes_array, bool force_empty_nulls,
+    bool zero_undefined_offsets, double sparsity, int64_t* out_max_view_end,
+    int64_t alignment, MemoryPool* memory_pool) {
+  using TypeClass = typename OffsetArrayType::TypeClass;
+
+  auto* sizes = mutable_sizes_array.data()->template GetMutableValues<offset_type>(1);
+
+  BufferVector buffers{2};
+  buffers[0] = NULLPTR;  // sizes can have nulls, offsets don't have to
+  buffers[1] = *AllocateBuffer(sizeof(offset_type) * mutable_sizes_array.length(),
+                               alignment, memory_pool);
+  auto offsets = buffers[1]->mutable_data_as<offset_type>();
+
+  double offset_base = 0.0;
+  offset_type max_view_end = 0;
+  for (int64_t i = 0; i < mutable_sizes_array.length(); ++i) {
+    const auto offset = static_cast<offset_type>(std::llround(offset_base));
+    if (mutable_sizes_array.IsNull(i)) {
+      if (force_empty_nulls) {
+        sizes[i] = 0;
+      }
+      offsets[i] = zero_undefined_offsets ? 0 : offset;
+    } else {
+      if (sizes[i] == 0) {
+        offsets[i] = zero_undefined_offsets ? 0 : offset;
+      } else {
+        offsets[i] = offset;
+        DCHECK_LT(offset, std::numeric_limits<offset_type>::max() - sizes[i]);
+        offset_base = std::max(0.0, offset_base + (sparsity * sizes[i]));
+      }
+    }
+    max_view_end = std::max(max_view_end, offsets[i] + sizes[i]);
+  }
+  *out_max_view_end = max_view_end;
+
+  auto array_data =
+      ArrayData::Make(TypeTraits<TypeClass>::type_singleton(),
+                      mutable_sizes_array.length(), std::move(buffers), /*null_count=*/0);
+  return std::make_shared<OffsetArrayType>(std::move(array_data));
+}
+
+template <typename ArrayType, typename RAG>
+Result<std::shared_ptr<Array>> ArrayOfListView(RAG& self, const Field& field,

Review Comment:
   It is indeed a problem, but note that regular list generators are also split in 2 versions. I started this PR with an unified version, but the code was too complicated (the parameterization is different for some reason) and after your first review I added the `random.cc: Simplify and split the random generator into two algorithms` commit.
   
   I mentioned this duplication in the issues about `concatenate.cc` also having yet another random generator: https://github.com/apache/arrow/issues/38656#issuecomment-1806507591



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {
+    for (int64_t i = 0; i < input.length; i++) {
+      MINIMIZE_MIN_VIEW_OFFSET(i);
+    }
+  } else {
+    SetBitRunReader reader(validity, input.offset, input.length);
+    while (true) {
+      const auto run = reader.NextRun();
+      if (run.length == 0) {
+        break;
+      }
+      for (int64_t i = run.position; i < run.position + run.length; ++i) {
+        MINIMIZE_MIN_VIEW_OFFSET(i);
+      }
+    }
+  }
+  return min_offset;
+
+#undef MINIMIZE_MIN_VIEW_OFFSET
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  const auto values_length = input.child_data[0].length;
+
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Early-exit: 64-bit overflow detected. This is not possible on a valid list-view,
+  // but we return the maximum possible value to avoid undefined behavior.

Review Comment:
   This is a public API. I can't guarantee on what kind of data it will be 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {

Review Comment:
   How can I early return if I find a `0`? (the most common 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -383,6 +393,12 @@ class NullArrayFactory {
       return Status::OK();
     }
 
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T&) {
+      buffer_length_ = length_ * sizeof(typename T::offset_type);

Review Comment:
   Yeah. I also thought it should have been length 0 on the recursive call to children. I'm pushing a change with some `XXX` comments we should remove before 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// value builder.
+  ///
+  /// \pre if is_valid is false, list_length MUST be 0
+  /// \param is_valid Whether the new list slot is valid
+  /// \param list_length The number of elements in the list
+  Status Append(bool is_valid, int64_t list_length) {

Review Comment:
   Not that this is the new parent class of both list and list-view builders. The classic list builders remain with their API `Append(bool)` is still supported.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the

Review Comment:
   This is called to "start a new variable-length list slot". The values added to the values builder before this call are not part of the newly started variable-length list slot.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,285 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.MayHaveNulls() ? input.buffers[0].data : nullptr;
+  const auto* offsets = reinterpret_cast<const offset_type*>(input.buffers[1].data);
+  const auto* sizes = reinterpret_cast<const offset_type*>(input.buffers[2].data);
+
+  // It's very likely that the first non-null non-empty list-view starts at
+  // offset 0 of the child array.
+  int64_t i = 0;
+  while (i < input.length && (input.IsNull(i) || sizes[input.offset + i] == 0)) {
+    i += 1;
+  }
+  if (i >= input.length) {
+    return 0;
+  }
+  auto min_offset = offsets[input.offset + i];
+  if (ARROW_PREDICT_TRUE(min_offset == 0)) {
+    // Early exit: offset 0 found already.
+    return 0;
+  }

Review Comment:
   I'm going to use an `std::optional`. This is ugly no matter what.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/c/bridge_test.cc:
##########
@@ -3631,6 +3824,13 @@ TEST_F(TestArrayRoundtrip, List) {
   TestWithJSONSliced(fixed_size_list(int32(), 3), "[[4, 5, 6], null, [7, 8, null]]");
 }
 
+TEST_F(TestArrayRoundtrip, ListView) {
+  TestWithJSON(list_view(int32()), "[]");
+  TestWithJSON(list_view(int32()), "[[4, 5], [6, null], null]");

Review Comment:
   Fair enough :-)



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));
+    if (ARROW_PREDICT_TRUE(i != j)) {
+      // Swap validity bits
+      if (validity) {
+        const bool valid_i = bit_util::GetBit(validity, data.offset + i);
+        const bool valid_j = bit_util::GetBit(validity, data.offset + i);
+        if (valid_i != valid_j) {
+          bit_util::SetBitTo(validity, data.offset + i, valid_j);
+          bit_util::SetBitTo(validity, data.offset + j, valid_i);
+        }
+      }
+      // Swap offsets and sizes
+      std::swap(offsets[i], offsets[j]);
+      std::swap(sizes[i], sizes[j]);
+    }
+  }
+}
+
+/// \brief Generate the list-view offsets based on a random buffer of sizes.
+///
+/// The sizes buffer is an input of this function, but when force_empty_nulls is true,
+/// some values on the sizes buffer can be set to 0.
+///
+/// When sparsity is 0.0, the list-view spans are perfectly packed one after the
+/// other. If sparsity is greater than 0.0, the list-view spans are set apart
+/// from each other in proportion to the sparsity value and size of each
+/// list-view. A negative sparsity means each list-view shares a fraction of the
+/// values used by the previous list-view.
+///
+/// For instance, a sparsity of -1.0 means the values array will only need enough values
+/// for the largest list-view with all the other list-views spanning some of these same
+/// values.
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] mutable_sizes_array The array of sizes to use
+/// \param[in] force_empty_nulls Whether to force null list-view sizes to be 0
+/// \param[in] zero_undefined_offsets Whether to zero the offsets of list-views that have
+/// 0 set as the size
+/// \param[in] sparsity The sparsity of the generated list-view offsets
+/// \param[out] out_max_view_end The maximum value of the end of a list-view
+template <typename OffsetArrayType, typename offset_type>
+std::shared_ptr<Array> ViewOffsetsFromLengthsArray(
+    SeedType seed, OffsetArrayType& mutable_sizes_array, bool force_empty_nulls,
+    bool zero_undefined_offsets, double sparsity, int64_t* out_max_view_end,
+    int64_t alignment, MemoryPool* memory_pool) {
+  using TypeClass = typename OffsetArrayType::TypeClass;
+
+  auto* sizes = mutable_sizes_array.data()->template GetMutableValues<offset_type>(1);
+
+  BufferVector buffers{2};
+  buffers[0] = NULLPTR;  // sizes can have nulls, offsets don't have to
+  buffers[1] = *AllocateBuffer(sizeof(offset_type) * mutable_sizes_array.length(),
+                               alignment, memory_pool);
+  auto offsets = buffers[1]->mutable_data_as<offset_type>();
+
+  double offset_base = 0.0;
+  offset_type max_view_end = 0;
+  for (int64_t i = 0; i < mutable_sizes_array.length(); ++i) {
+    const auto offset = static_cast<offset_type>(std::llround(offset_base));
+    if (mutable_sizes_array.IsNull(i)) {
+      if (force_empty_nulls) {
+        sizes[i] = 0;
+      }
+      offsets[i] = zero_undefined_offsets ? 0 : offset;
+    } else {
+      if (sizes[i] == 0) {
+        offsets[i] = zero_undefined_offsets ? 0 : offset;
+      } else {
+        offsets[i] = offset;
+        DCHECK_LT(offset, std::numeric_limits<offset_type>::max() - sizes[i]);
+        offset_base = std::max(0.0, offset_base + (sparsity * sizes[i]));
+      }
+    }
+    max_view_end = std::max(max_view_end, offsets[i] + sizes[i]);
+  }
+  *out_max_view_end = max_view_end;
+
+  auto array_data =
+      ArrayData::Make(TypeTraits<TypeClass>::type_singleton(),
+                      mutable_sizes_array.length(), std::move(buffers), /*null_count=*/0);
+  return std::make_shared<OffsetArrayType>(std::move(array_data));
+}
+
+template <typename ArrayType, typename RAG>
+Result<std::shared_ptr<Array>> ArrayOfListView(RAG& self, const Field& field,

Review Comment:
   I'm removing the `sparsity` parameter, but adding coverage to this makes controlling the size of the list-views an issue for benchmarks (that want sizes to follow a certain distribution of sizes).



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  auto is_null_or_empty = [&](int64_t i) {
+    return (validity && !bit_util::GetBit(validity, list_view_array.offset() + i)) ||
+           sizes[i] == 0;
+  };
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid, non-empty list-view and last offset
+  // of the current contiguous fragment in values.
+  constexpr int64_t kUninitialized = -1;
+  int64_t first_i = kUninitialized;
+  offset_type end_offset;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    if (offsets[i] == end_offset) {
+      end_offset += sizes[i];
+      continue;
+    }
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+  }
+  if (first_i != kUninitialized) {
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+  }
+
+  // Final attempt to avoid invoking Concatenate().
+  if (non_null_fragments.size() == 1) {
+    return non_null_fragments[0];
+  } else if (non_null_fragments.size() == 0) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  return Concatenate(non_null_fragments, memory_pool);
+}
+
 std::shared_ptr<Array> BoxOffsets(const std::shared_ptr<DataType>& boxed_type,
                                   const ArrayData& data) {
+  const int64_t num_offsets =
+      is_list_view(data.type->id()) ? data.length : data.length + 1;
   std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[1]};
   auto offsets_data =
-      std::make_shared<ArrayData>(boxed_type, data.length + 1, std::move(buffers),
+      std::make_shared<ArrayData>(boxed_type, /*length=*/num_offsets, std::move(buffers),
+                                  /*null_count=*/0, data.offset);
+  return MakeArray(offsets_data);
+}
+
+std::shared_ptr<Array> BoxSizes(const std::shared_ptr<DataType>& boxed_type,
+                                const ArrayData& data) {
+  DCHECK(is_list_view(data.type->id()));
+  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[2]};
+  auto offsets_data =

Review Comment:
   Rename to `sizes_data`?



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the
-  /// value builder
-  Status Append(bool is_valid = true) {
+  /// value builder.
+  ///
+  /// \pre if is_valid is false, list_length MUST be 0
+  /// \param is_valid Whether the new list slot is valid
+  /// \param list_length The number of elements in the list
+  Status Append(bool is_valid, int64_t list_length) {

Review Comment:
   This would be useful, can we add this in comment? Since the interface would be a bit tricky.



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,57 +713,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {

Review Comment:
   So this is for List rather than ListView?



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -137,6 +137,77 @@ Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListArrayFromArray
   return std::make_shared<ArrayType>(std::move(data));
 }
 
+template <typename TYPE>
+Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListViewArrayFromArrays(
+    std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+    const Array& values, MemoryPool* pool, std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+    int64_t null_count = kUnknownNullCount) {
+  using offset_type = typename TYPE::offset_type;
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+  if (offsets.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List offsets must be ", OffsetArrowType::type_name());
+  }
+
+  if (sizes.length() != offsets.length() && sizes.length() != offsets.length() - 1) {
+    return Status::Invalid(
+        "List sizes must have the same length as offsets or one less than offsets");
+  }
+  if (sizes.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List sizes must be ", OffsetArrowType::type_name());
+  }
+
+  if (offsets.offset() != sizes.offset()) {
+    return Status::Invalid("List offsets and sizes must have the same offset");
+  }

Review Comment:
   It's now noted in the docstrings. (meaning: I will push my local commit 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  auto is_null_or_empty = [&](int64_t i) {
+    return (validity && !bit_util::GetBit(validity, list_view_array.offset() + i)) ||
+           sizes[i] == 0;
+  };
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid, non-empty list-view and last offset
+  // of the current contiguous fragment in values.
+  constexpr int64_t kUninitialized = -1;
+  int64_t first_i = kUninitialized;
+  offset_type end_offset;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    if (offsets[i] == end_offset) {
+      end_offset += sizes[i];
+      continue;
+    }
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+  }
+  if (first_i != kUninitialized) {
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+  }
+
+  // Final attempt to avoid invoking Concatenate().
+  if (non_null_fragments.size() == 1) {
+    return non_null_fragments[0];
+  } else if (non_null_fragments.size() == 0) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  return Concatenate(non_null_fragments, memory_pool);
+}
+
 std::shared_ptr<Array> BoxOffsets(const std::shared_ptr<DataType>& boxed_type,
                                   const ArrayData& data) {
+  const int64_t num_offsets =
+      is_list_view(data.type->id()) ? data.length : data.length + 1;
   std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[1]};
   auto offsets_data =
-      std::make_shared<ArrayData>(boxed_type, data.length + 1, std::move(buffers),
+      std::make_shared<ArrayData>(boxed_type, /*length=*/num_offsets, std::move(buffers),
+                                  /*null_count=*/0, data.offset);
+  return MakeArray(offsets_data);
+}
+
+std::shared_ptr<Array> BoxSizes(const std::shared_ptr<DataType>& boxed_type,
+                                const ArrayData& data) {
+  DCHECK(is_list_view(data.type->id()));
+  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[2]};
+  auto offsets_data =

Review Comment:
   This is a good rename. I'm applying it. It's OK to pass sizes to `SwapOffsets` — it should be obvious to the reader that swapping endianness of sizes is the same operation as swapping the endianness of offsets.



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +260,113 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0, 0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) has disjoint views which completely cover the values array
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  auto is_null_or_empty = [&](int64_t i) {
+    return (validity && !bit_util::GetBit(validity, list_view_array.offset() + i)) ||
+           sizes[i] == 0;
+  };
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid, non-empty list-view and last offset
+  // of the current contiguous fragment in values.
+  constexpr int64_t kUninitialized = -1;
+  int64_t first_i = kUninitialized;
+  offset_type end_offset;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {
+    if (is_null_or_empty(i)) continue;
+
+    if (offsets[i] == end_offset) {
+      end_offset += sizes[i];
+      continue;
+    }
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+  }
+  if (first_i != kUninitialized) {
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+  }
+
+  // Final attempt to avoid invoking Concatenate().
+  if (non_null_fragments.size() == 1) {
+    return non_null_fragments[0];
+  } else if (non_null_fragments.size() == 0) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  return Concatenate(non_null_fragments, memory_pool);
+}
+
 std::shared_ptr<Array> BoxOffsets(const std::shared_ptr<DataType>& boxed_type,
                                   const ArrayData& data) {
+  const int64_t num_offsets =
+      is_list_view(data.type->id()) ? data.length : data.length + 1;
   std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[1]};
   auto offsets_data =
-      std::make_shared<ArrayData>(boxed_type, data.length + 1, std::move(buffers),
+      std::make_shared<ArrayData>(boxed_type, /*length=*/num_offsets, std::move(buffers),
+                                  /*null_count=*/0, data.offset);
+  return MakeArray(offsets_data);
+}
+
+std::shared_ptr<Array> BoxSizes(const std::shared_ptr<DataType>& boxed_type,
+                                const ArrayData& data) {
+  DCHECK(is_list_view(data.type->id()));
+  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, data.buffers[2]};
+  auto offsets_data =

Review Comment:
   This is a good rename. I'm applying it. It's OK to pass sizes to `SwapOffsets` — it should be obvious to the reader that swapping endianness of sizes is the same operation as swapping the endianness of offsets.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -137,6 +137,77 @@ Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListArrayFromArray
   return std::make_shared<ArrayType>(std::move(data));
 }
 
+template <typename TYPE>
+Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListViewArrayFromArrays(
+    std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+    const Array& values, MemoryPool* pool, std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+    int64_t null_count = kUnknownNullCount) {
+  using offset_type = typename TYPE::offset_type;
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+  if (offsets.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List offsets must be ", OffsetArrowType::type_name());
+  }
+
+  if (sizes.length() != offsets.length() && sizes.length() != offsets.length() - 1) {
+    return Status::Invalid(
+        "List sizes must have the same length as offsets or one less than offsets");
+  }
+  if (sizes.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List sizes must be ", OffsetArrowType::type_name());
+  }
+
+  if (offsets.offset() != sizes.offset()) {
+    return Status::Invalid("List offsets and sizes must have the same offset");
+  }

Review Comment:
   It's now noted in the docstrings.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

Posted by "conbench-apache-arrow[bot] (via GitHub)" <gi...@apache.org>.
conbench-apache-arrow[bot] commented on PR #35345:
URL: https://github.com/apache/arrow/pull/35345#issuecomment-1823401181

   After merging your PR, Conbench analyzed the 6 benchmarking runs that have been run so far on merge-commit 8cc71abc9a061f6a9e8a29385f4e5d87bd6890d8.
   
   There were 5 benchmark results indicating a performance regression:
   
   - Commit Run on `ursa-i9-9960x` at [2023-11-22 16:59:04Z](https://conbench.ursa.dev/compare/runs/5cc8ab141ec444cfa451ab160379a33c...8b23250fee75448f923950cbf65170a2/)
     - [`file-write` (Python) with compression=uncompressed, dataset=fanniemae_2016Q4, file_type=parquet, input_type=dataframe](https://conbench.ursa.dev/compare/benchmarks/0655d28de5bf72cd80009ab2f1d302cc...0655e3c54c627ead8000928302dc9f72)
     - [`dataset-selectivity` (Python) with dataset=nyctaxi_multi_ipc_s3, selectivity=10%](https://conbench.ursa.dev/compare/benchmarks/0655d21750007cd8800085c7241621d5...0655e34fe7b87df480000086429726a5)
   - and 3 more (see the report linked below)
   
   The [full Conbench report](https://github.com/apache/arrow/runs/18944567991) has more details. It also includes information about 14 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   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] bkietz commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset

Review Comment:
   ```suggestion
     // Index of first valid, non-empty list-view and last offset
   ```



##########
cpp/src/arrow/util/list_util_test.cc:
##########
@@ -0,0 +1,228 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <gtest/gtest.h>
+
+#include "arrow/array/builder_nested.h"
+#include "arrow/util/list_util.h"
+
+#include "arrow/testing/builder.h"
+#include "arrow/testing/gtest_util.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+using ListAndListViewTypes =
+    ::testing::Types<ListType, LargeListType, ListViewType, LargeListViewType>;
+
+template <typename T>
+class TestListUtils : public ::testing::Test {
+ public:
+  using TypeClass = T;
+  using offset_type = typename TypeClass::offset_type;
+  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
+
+  void SetUp() override {
+    value_type_ = int16();
+    type_ = std::make_shared<T>(value_type_);
+
+    std::unique_ptr<ArrayBuilder> tmp;
+    ASSERT_OK(MakeBuilder(pool_, type_, &tmp));
+    builder_.reset(checked_cast<BuilderType*>(tmp.release()));
+  }
+
+  void TestRangeOfValuesUsed() {
+    std::shared_ptr<ArrayData> result;
+
+    // Empty list-like array
+    ASSERT_OK(builder_->FinishInternal(&result));

Review Comment:
   Why use a builder here instead of ArrayFromJSON?



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint

Review Comment:
   nit: "disjoint" by itself admits a list view array with views from 0->3 and 5->7, which wouldn't be compatible with this optimization. The code correctly checks for full cover
   ```suggestion
     //   3) has disjoint views which completely cover the values array
   ```



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -321,6 +497,123 @@ std::shared_ptr<Array> LargeListArray::offsets() const {
   return BoxOffsets(int64(), *data_);
 }
 
+// ----------------------------------------------------------------------
+// ListViewArray
+
+ListViewArray::ListViewArray(std::shared_ptr<ArrayData> data) {
+  ListViewArray::SetData(std::move(data));
+}
+
+ListViewArray::ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                             std::shared_ptr<Buffer> value_offsets,
+                             std::shared_ptr<Buffer> value_sizes,
+                             std::shared_ptr<Array> values,
+                             std::shared_ptr<Buffer> null_bitmap, int64_t null_count,
+                             int64_t offset) {
+  ListViewArray::SetData(ArrayData::Make(
+      type, length,

Review Comment:
   ```suggestion
         std::move(type), length,
   ```



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,188 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };

Review Comment:
   Is there a reason to do this instead of providing factories for the corresponding failure Status?
   
   ```
   static Status OutOfBoundsListViewOffset(int64_t offset, int64_t slot) {
     // ...
   }
   static Status OutOfBoundsListViewSize(int64_t size, int64_t slot) {
     // ...
   }
   ```



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    if (offsets[i] == end_offset) {
+      end_offset += sizes[i];
+    } else {
+      non_null_fragments.push_back(
+          SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+      first_i = i;
+      end_offset = offsets[i] + sizes[i];
+    }
+  }
+  if (first_i >= 0) {
+    non_null_fragments.push_back(
+        SliceArrayWithOffsets(*value_array, offsets[first_i], end_offset));
+  }
+
+  // Final attempt to avoid invoking Concatenate().
+  if (non_null_fragments.size() == 1) {
+    return non_null_fragments[0];
+  } else if (non_null_fragments.size() == 0) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  return Concatenate(non_null_fragments, memory_pool);

Review Comment:
   Would you mind moving the single chunk special case into Concatenate? It's a cheap fast path which is useful for data of any type
   
   ```suggestion
     // Concatenate() doesn't support an empty vector
     if (non_null_fragments.size() == 0) {
       return MakeEmptyArray(value_array->type(), memory_pool);
     }
     return Concatenate(non_null_fragments, memory_pool);
   ```



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -137,6 +137,78 @@ Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListArrayFromArray
   return std::make_shared<ArrayType>(std::move(data));
 }
 
+template <typename TYPE>
+Result<std::shared_ptr<typename TypeTraits<TYPE>::ArrayType>> ListViewArrayFromArrays(
+    std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+    const Array& values, MemoryPool* pool, std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+    int64_t null_count = kUnknownNullCount) {
+  using offset_type = typename TYPE::offset_type;
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+  if (offsets.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List offsets must be ", OffsetArrowType::type_name());
+  }
+
+  if (sizes.length() != offsets.length() && sizes.length() != offsets.length() - 1) {
+    return Status::Invalid(
+        "List sizes must have the same length as offsets or one less than offsets");
+  }
+  if (sizes.type_id() != OffsetArrowType::type_id) {
+    return Status::TypeError("List sizes must be ", OffsetArrowType::type_name());
+  }
+
+  if (offsets.offset() != sizes.offset()) {
+    return Status::Invalid("List offsets and sizes must have the same offset");
+  }
+  const int64_t offset = sizes.offset();
+
+  if (null_bitmap) {
+    if (offsets.null_count() > 0 || sizes.null_count() > 0) {
+      return Status::Invalid(
+          "Ambiguous to specify both validity map and offsets or sizes with nulls");
+    }
+    if (offset != 0) {
+      return Status::Invalid(
+          "List offsets and sizes must not be slices if a validity map is specified");
+    }
+  } else {
+    if (offsets.null_count() > 0 && sizes.null_count() > 0) {
+      return Status::Invalid("Ambiguous to specify both offsets and sizes with nulls");
+    }
+  }
+
+  DCHECK(offsets.length() == sizes.length() || offsets.length() - 1 == sizes.length());
+
+  using OffsetArrayType = typename TypeTraits<OffsetArrowType>::ArrayType;
+  const auto& typed_offsets = checked_cast<const OffsetArrayType&>(offsets);
+  const auto& typed_sizes = checked_cast<const OffsetArrayType&>(sizes);
+
+  auto derived_validity_buffer = std::move(null_bitmap);
+  int64_t null_count_ = null_count;
+  if (offsets.null_count() > 0) {
+    derived_validity_buffer = offsets.null_bitmap();
+    null_count_ = offsets.null_count();
+    // We allow construction from an offsets array containing one extra value.
+    // If that is the case, we might need to discount one null from out_null_count.
+    if (offsets.length() - 1 == sizes.length() && !offsets.IsValid(sizes.length())) {
+      null_count_ -= 1;

Review Comment:
   Why introduce another variable?
   ```suggestion
     if (offsets.null_count() > 0) {
       derived_validity_buffer = offsets.null_bitmap();
       null_count = offsets.null_count();
       // We allow construction from an offsets array containing one extra value.
       // If that is the case, we might need to discount one null from out_null_count.
       if (offsets.length() - 1 == sizes.length() && !offsets.IsValid(sizes.length())) {
         null_count -= 1;
   ```



##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {

Review Comment:
   Is it a worthwhile special case to search for the first list with a specialized loop? It'd be easier to read if the two loops were consolidated, and the special case I'm used to seeing is explicitly hoisting a no-nulls case



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,188 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  std::pair<ListViewValidationError, int64_t> DoFullyValidateOffsetsAndSizes(
+      int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return {kOutOfBoundsOffset, slot};
+              }
+              if (offset + size > offset_limit) {

Review Comment:
   Since we haven't checked what `size` is yet, it might contain `offset_type::MAX` or something else which would provoke overflow



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,188 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  std::pair<ListViewValidationError, int64_t> DoFullyValidateOffsetsAndSizes(
+      int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return {kOutOfBoundsOffset, slot};
+              }
+              if (offset + size > offset_limit) {
+                return {kOutOfBoundsSize, slot};
+              }
+            } else if (size < 0) {
+              return {kOutOfBoundsSize, slot};
+            }
+          }
+        }
+      }
+    } else {
+      for (; slot < data.length; slot++) {
+        const auto size = sizes[slot];
+        if (size > 0) {
+          const auto offset = offsets[slot];
+          if (offset < 0 || offset > offset_limit) {
+            return {kOutOfBoundsOffset, slot};
+          }
+          if (offset + size > offset_limit) {
+            return {kOutOfBoundsSize, slot};
+          }
+        } else if (size < 0) {
+          return {kOutOfBoundsSize, slot};
+        }
+      }
+    }
+
+    return {kOk, slot};
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    auto [result, slot] = DoFullyValidateOffsetsAndSizes<offset_type>(offset_limit);
+    switch (result) {
+      case kOk:
+        break;
+      case kOutOfBoundsOffset: {
+        const auto offset = offsets[slot];
+        return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                               " out of bounds: ", offset,
+                               offset < 0 ? " < 0" : " > offset_limit");

Review Comment:
   ```suggestion
                                  " out of bounds. Expected ", offset, " to be at least ",
                                  "0 and less than ", offset_limit);
   ```



##########
cpp/src/arrow/builder.cc:
##########
@@ -219,6 +219,20 @@ struct MakeBuilderImpl {
     return Status::OK();
   }
 
+  Status Visit(const ListViewType& list_view_type) {
+    std::shared_ptr<DataType> value_type = list_view_type.value_type();
+    ARROW_ASSIGN_OR_RAISE(auto value_builder, ChildBuilder(value_type));
+    out.reset(new ListViewBuilder(pool, std::move(value_builder), type));

Review Comment:
   ```suggestion
       out.reset(new ListViewBuilder(pool, std::move(value_builder), std::move(type)));
   ```



##########
cpp/src/arrow/array/builder_base.cc:
##########
@@ -319,10 +324,30 @@ struct DerefConstIterator {
   pointer operator->() const { return &(**it); }
 };
 
+/// If A and B are equivalent types, a builder of type A can receive
+/// scalar values of type B and a builder of type B can receive
+/// scalar values of type A.
+///
+/// \param a Type A.
+/// \param b Type B.
+bool AreScalarTypesEquivalent(const DataType& a, const DataType& b) {
+  if (a.Equals(b)) {
+    return true;
+  }
+  if ((a.id() == Type::LIST && b.id() == Type::LIST_VIEW) ||
+      (a.id() == Type::LIST_VIEW && b.id() == Type::LIST) ||
+      (a.id() == Type::LARGE_LIST && b.id() == Type::LARGE_LIST_VIEW) ||
+      (a.id() == Type::LARGE_LIST_VIEW && b.id() == Type::LARGE_LIST)) {
+    return checked_cast<const BaseListType&>(a).value_type()->Equals(
+        *checked_cast<const BaseListType&>(b).value_type());
+  }
+  return false;
+}
+
 }  // namespace
 
 Status ArrayBuilder::AppendScalar(const Scalar& scalar, int64_t n_repeats) {
-  if (!scalar.type->Equals(type())) {
+  if (!AreScalarTypesEquivalent(*scalar.type, *type())) {

Review Comment:
   Where is this used? I'm not sure I like the idea of introducing implicit conversions. For one thing, what if the builder is `list(field("item", int8(), /*nullable=*/false))` but the appended scalar is `list(int8())`? The implicit conversion would drop the guarantee of no nulls from the built 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,188 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  std::pair<ListViewValidationError, int64_t> DoFullyValidateOffsetsAndSizes(
+      int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return {kOutOfBoundsOffset, slot};
+              }
+              if (offset + size > offset_limit) {
+                return {kOutOfBoundsSize, slot};
+              }
+            } else if (size < 0) {
+              return {kOutOfBoundsSize, slot};
+            }
+          }
+        }
+      }
+    } else {
+      for (; slot < data.length; slot++) {
+        const auto size = sizes[slot];
+        if (size > 0) {
+          const auto offset = offsets[slot];
+          if (offset < 0 || offset > offset_limit) {
+            return {kOutOfBoundsOffset, slot};
+          }
+          if (offset + size > offset_limit) {
+            return {kOutOfBoundsSize, slot};
+          }
+        } else if (size < 0) {
+          return {kOutOfBoundsSize, slot};
+        }
+      }
+    }
+
+    return {kOk, slot};
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    auto [result, slot] = DoFullyValidateOffsetsAndSizes<offset_type>(offset_limit);
+    switch (result) {
+      case kOk:
+        break;
+      case kOutOfBoundsOffset: {
+        const auto offset = offsets[slot];
+        return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                               " out of bounds: ", offset,
+                               offset < 0 ? " < 0" : " > offset_limit");

Review Comment:
   Done in a fixup commit.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_base.cc:
##########
@@ -319,10 +324,30 @@ struct DerefConstIterator {
   pointer operator->() const { return &(**it); }
 };
 
+/// If A and B are equivalent types, a builder of type A can receive
+/// scalar values of type B and a builder of type B can receive
+/// scalar values of type A.
+///
+/// \param a Type A.
+/// \param b Type B.
+bool AreScalarTypesEquivalent(const DataType& a, const DataType& b) {
+  if (a.Equals(b)) {
+    return true;
+  }
+  if ((a.id() == Type::LIST && b.id() == Type::LIST_VIEW) ||
+      (a.id() == Type::LIST_VIEW && b.id() == Type::LIST) ||
+      (a.id() == Type::LARGE_LIST && b.id() == Type::LARGE_LIST_VIEW) ||
+      (a.id() == Type::LARGE_LIST_VIEW && b.id() == Type::LARGE_LIST)) {
+    return checked_cast<const BaseListType&>(a).value_type()->Equals(
+        *checked_cast<const BaseListType&>(b).value_type());
+  }
+  return false;
+}
+
 }  // namespace
 
 Status ArrayBuilder::AppendScalar(const Scalar& scalar, int64_t n_repeats) {
-  if (!scalar.type->Equals(type())) {
+  if (!AreScalarTypesEquivalent(*scalar.type, *type())) {

Review Comment:
   I added a commit that reverts these changes. All tests passes. I might float this type equivalence idea in the future when I can build a case for 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {

Review Comment:
   ~I can extract a few more functions. Just a minute.~
   
   nvm, I think this is the more straightforward solution.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.cc:
##########
@@ -189,11 +261,109 @@ Result<std::shared_ptr<Array>> FlattenListArray(const ListArrayT& list_array,
   return Concatenate(non_null_fragments, memory_pool);
 }
 
+template <typename ListViewArrayT>
+Result<std::shared_ptr<Array>> FlattenListViewArray(const ListViewArrayT& list_view_array,
+                                                    MemoryPool* memory_pool) {
+  using offset_type = typename ListViewArrayT::offset_type;
+  const int64_t list_view_array_length = list_view_array.length();
+  std::shared_ptr<arrow::Array> value_array = list_view_array.values();
+
+  if (list_view_array_length == 0) {
+    return SliceArrayWithOffsets(*value_array, 0, 0);
+  }
+
+  // If the list array is *all* nulls, then just return an empty array.
+  if (list_view_array.null_count() == list_view_array.length()) {
+    return MakeEmptyArray(value_array->type(), memory_pool);
+  }
+
+  const auto* validity = list_view_array.data()->template GetValues<uint8_t>(0);
+  const auto* offsets = list_view_array.data()->template GetValues<offset_type>(1);
+  const auto* sizes = list_view_array.data()->template GetValues<offset_type>(2);
+
+  // If a ListViewArray:
+  //
+  //   1) does not contain nulls
+  //   2) has sorted offsets
+  //   3) every view is disjoint
+  //
+  // then simply slice its value array with the first offset and end of the last list
+  // view.
+  if (list_view_array.null_count() == 0) {
+    bool sorted_and_disjoint = true;
+    for (int64_t i = 1; sorted_and_disjoint && i < list_view_array_length; ++i) {
+      sorted_and_disjoint &=
+          sizes[i - 1] == 0 || offsets[i] - offsets[i - 1] == sizes[i - 1];
+    }
+
+    if (sorted_and_disjoint) {
+      const auto begin_offset = list_view_array.value_offset(0);
+      const auto end_offset = list_view_array.value_offset(list_view_array_length - 1) +
+                              list_view_array.value_length(list_view_array_length - 1);
+      return SliceArrayWithOffsets(*value_array, begin_offset, end_offset);
+    }
+  }
+
+  std::vector<std::shared_ptr<Array>> non_null_fragments;
+  // Index of first valid list-view and last offset
+  // of the current contiguous fragment in values.
+  int64_t first_i = -1;
+  offset_type end_offset = -1;
+  int64_t i = 0;
+  for (; i < list_view_array_length; i++) {
+    if ((validity && !bit_util::GetBit(validity, i)) || sizes[i] == 0) {
+      continue;
+    }
+    first_i = i;
+    end_offset = offsets[i] + sizes[i];
+    break;
+  }
+  i += 1;
+  for (; i < list_view_array_length; i++) {

Review Comment:
   I tried to write this with a single loop, but separating like this allows the second loop to build up on the invariant that `last_offset` is set which makes it much cleaner and easier to understand.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -248,6 +305,38 @@ class ConcatenateImpl {
     return ConcatenateImpl(child_data, pool_).Concatenate(&out_->child_data[0]);
   }
 
+  template <typename T>
+  enable_if_list_view<T, Status> Visit(const T& type) {
+    using offset_type = typename T::offset_type;
+    out_->buffers.resize(3);
+    out_->child_data.resize(1);
+
+    // Calculate the ranges of values that each list-view array uses
+    std::vector<Range> value_ranges;
+    value_ranges.reserve(in_.size());
+    for (const auto& input : in_) {
+      ArraySpan input_span(*input);
+      Range range;
+      ARROW_ASSIGN_OR_RAISE(std::tie(range.offset, range.length),
+                            list_util::internal::RangeOfValuesUsed(input_span));
+      value_ranges.push_back(range);

Review Comment:
   I don't mind the explicitness of `std::tie(range.offset, range.length)` as pairs (which I use a lot) have the problem that you need to understand the context to be sure about the meaning of the `.second` field — is it a length or the inclusive offset bound.



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -248,6 +305,38 @@ class ConcatenateImpl {
     return ConcatenateImpl(child_data, pool_).Concatenate(&out_->child_data[0]);
   }
 
+  template <typename T>
+  enable_if_list_view<T, Status> Visit(const T& type) {
+    using offset_type = typename T::offset_type;
+    out_->buffers.resize(3);
+    out_->child_data.resize(1);
+
+    // Calculate the ranges of values that each list-view array uses
+    std::vector<Range> value_ranges;
+    value_ranges.reserve(in_.size());
+    for (const auto& input : in_) {
+      ArraySpan input_span(*input);
+      Range range;
+      ARROW_ASSIGN_OR_RAISE(std::tie(range.offset, range.length),
+                            list_util::internal::RangeOfValuesUsed(input_span));
+      value_ranges.push_back(range);

Review Comment:
   I don't mind the explicitness of `std::tie(range.offset, range.length)` as pairs (which I use a lot) have the problem that you need to understand the context to be sure about the meaning of the `.second` field — is it a length or the inclusive offset bound.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -602,8 +691,11 @@ class ConcatenateImpl {
 }  // namespace
 
 Result<std::shared_ptr<Array>> Concatenate(const ArrayVector& arrays, MemoryPool* pool) {
-  if (arrays.size() == 0) {
-    return Status::Invalid("Must pass at least one array");
+  switch (arrays.size()) {
+    case 0:
+      return Status::Invalid("Must pass at least one array");
+    case 1:
+      return arrays[0];

Review Comment:
   I'm keeping the original code in `array_nested.cc` as `FlattenListArray` can totally return non-trimmed slices of arrays.
   
   ```cpp
     // Final attempt to avoid invoking Concatenate().
     if (non_null_fragments.size() == 1) {
       return non_null_fragments[0];
     } else if (non_null_fragments.size() == 0) {
       return MakeEmptyArray(value_array->type(), memory_pool);
     }
   
     return Concatenate(non_null_fragments, memory_pool);
   ```
   
   So my change was just reverting the commit that added the fast path 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,173 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > offset_limit");
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+              }
+              if (size > offset_limit - offset) {
+                return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+              }
+            } else if (size < 0) {
+              return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+            }
+          }
+        }
+      }
+    } else {
+      for (; slot < data.length; slot++) {
+        const auto size = sizes[slot];
+        if (size > 0) {
+          const auto offset = offsets[slot];
+          if (offset < 0 || offset > offset_limit) {
+            return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+          }
+          if (size > offset_limit - offset) {
+            return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+          }
+        } else if (size < 0) {
+          return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+        }
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
+    const bool non_empty = data.length > 0;
     if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+      // For length 0, an empty offsets buffer is accepted (ARROW-544).

Review Comment:
   But I can start being more strict and wait for it. Adding a commit.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -203,6 +203,54 @@ TEST_F(ConcatenateTest, LargeListType) {
   });
 }
 
+TEST_F(ConcatenateTest, ListViewType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    auto values_size = size * 4;
+    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
+
+    std::shared_ptr<Array> offsets;
+    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    offsets_vector.front() = 0;
+    offsets_vector.back() = values_size;
+    ArrayFromVector<Int32Type>(offsets_vector, &offsets);
+
+    std::shared_ptr<Array> sizes;
+    std::vector<int32_t> sizes_vector;
+    sizes_vector.reserve(size);
+    for (int32_t i = 0; i < size; ++i) {
+      sizes_vector.push_back(offsets_vector[i + 1] - offsets_vector[i]);

Review Comment:
   Done locally. Tests passing.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/scalar_test.cc:
##########
@@ -394,6 +394,10 @@ class TestRealScalar : public ::testing::Test {
 
   void TestLargeListOf() { TestListOf<LargeListScalar>(large_list(type_)); }
 
+  void TestListViewOf() { TestListOf<ListViewScalar>(list_view(type_)); }
+
+  void TestLargeListViewOf() { TestListOf<LargeListViewScalar>(large_list_view(type_)); }

Review Comment:
   I missed a list of types here, but after adding the list-view types, `MakeNullScalar` gets tested by `CheckMakeNullScalar`.
   
   ![image](https://github.com/apache/arrow/assets/207795/7cb5236c-f3d3-490b-b5d6-bcf6578952a3)
   



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -43,30 +43,31 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// ListArray
+// VarLengthListLikeArray
 
 template <typename TYPE>
-class BaseListArray;
+class VarLengthListLikeArray;

Review Comment:
   Waiting for an answer of my question on the thread 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,173 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > offset_limit");
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+              }
+              if (size > offset_limit - offset) {
+                return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+              }
+            } else if (size < 0) {
+              return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+            }
+          }
+        }
+      }
+    } else {
+      for (; slot < data.length; slot++) {
+        const auto size = sizes[slot];
+        if (size > 0) {
+          const auto offset = offsets[slot];
+          if (offset < 0 || offset > offset_limit) {
+            return OutOfBoundsListViewOffset<offset_type>(slot, offset_limit);
+          }
+          if (size > offset_limit - offset) {
+            return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+          }
+        } else if (size < 0) {
+          return OutOfBoundsListViewSize<offset_type>(slot, offset_limit);
+        }
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
+    const bool non_empty = data.length > 0;
     if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+      // For length 0, an empty offsets buffer is accepted (ARROW-544).

Review Comment:
   Note that `Buffer::Wrap(zero_len_vector)` can lead to a `Buffer` containing `nullptr` or an allocated value depending on whether the `std::vector` has increased in capacity or not (e.g. with a `.reserve()` call).



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   > Well, it's probably too late for 14.0.0 regardless, since @raulcd has already cut the release branch.
   
   Sure, but I still want to merge 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -666,12 +682,29 @@ class RepeatedArrayFactory {
     std::shared_ptr<Buffer> offsets_buffer;
     auto size = static_cast<typename T::offset_type>(value->length());
     RETURN_NOT_OK(CreateOffsetsBuffer(size, &offsets_buffer));
-
     out_ =
         std::make_shared<ArrayType>(scalar_.type, length_, offsets_buffer, value_array);
     return Status::OK();
   }
 
+  template <typename T>
+  enable_if_list_view<T, Status> Visit(const T& type) {
+    using ScalarType = typename TypeTraits<T>::ScalarType;
+    using ArrayType = typename TypeTraits<T>::ArrayType;
+
+    auto value = checked_cast<const ScalarType&>(scalar_).value;
+
+    auto size = static_cast<typename T::offset_type>(value->length());
+    BufferVector buffers(3);

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -159,16 +166,67 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+void PutListViewOffsets(const Buffer& src, offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const BufferVector& buffers,
+                                  const std::vector<Range>& value_ranges,
+                                  MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  const int64_t out_size = SumBufferSizes(buffers);
+  if (out_size > std::numeric_limits<offset_type>::max()) {
+    return Status::Invalid("offset overflow while concatenating arrays");
+  }

Review Comment:
   This doesn't seem right; we could have a ListView array with `2**34` empty lists even though that number is greater than the max of offset type. I think this check needs to inspect `value_ranges`, which can tell us what the maximum offset would need to be



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -159,16 +166,67 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+void PutListViewOffsets(const Buffer& src, offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const BufferVector& buffers,
+                                  const std::vector<Range>& value_ranges,
+                                  MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  const int64_t out_size = SumBufferSizes(buffers);
+  if (out_size > std::numeric_limits<offset_type>::max()) {
+    return Status::Invalid("offset overflow while concatenating arrays");
+  }
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(out_size, pool));
+  auto* out_data = reinterpret_cast<offset_type*>((*out)->mutable_data());

Review Comment:
   ```suggestion
     auto* out_data = (*out)->mutable_data_as<offset_type>();
   ```



##########
cpp/src/arrow/testing/random.cc:
##########
@@ -594,6 +595,217 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value));

Review Comment:
   Nice helper
   ```suggestion
       ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ", ArrowType::type_name()));
   ```



##########
cpp/src/arrow/ipc/writer.cc:
##########
@@ -430,6 +492,36 @@ class RecordBatchSerializer {
     return Status::OK();
   }
 
+  template <typename T>
+  enable_if_list_view<typename T::TypeClass, Status> Visit(const T& array) {
+    using offset_type = typename T::offset_type;
+
+    offset_type min_offset = 0;
+    offset_type max_end = 0;
+    {
+      std::shared_ptr<Buffer> value_offsets;
+      RETURN_NOT_OK(
+          GetZeroBasedListViewOffsets<T>(array, &value_offsets, &min_offset, &max_end));
+      out_->body_buffers.push_back(std::move(value_offsets));
+    }
+    {
+      std::shared_ptr<Buffer> value_sizes;
+      RETURN_NOT_OK(GetListViewSizes<T>(array, &value_sizes));
+      out_->body_buffers.push_back(std::move(value_sizes));
+    }
+
+    --max_recursion_depth_;

Review Comment:
   Nit: please move this to just above the VisitArray call



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -80,100 +89,91 @@ class BaseListBuilder : public ArrayBuilder {
     value_builder_->Reset();
   }
 
-  /// \brief Vector append
-  ///
-  /// If passed, valid_bytes is of equal length to values, and any zero byte
-  /// will be considered as a null for that slot
-  Status AppendValues(const offset_type* offsets, int64_t length,
-                      const uint8_t* valid_bytes = NULLPTR) {
-    ARROW_RETURN_NOT_OK(Reserve(length));
-    UnsafeAppendToBitmap(valid_bytes, length);
-    offsets_builder_.UnsafeAppend(offsets, length);
-    return Status::OK();
-  }
-
   /// \brief Start a new variable-length list slot
   ///
   /// This function should be called before beginning to append elements to the

Review Comment:
   That seems fine, then let's rewrite the comment as "Items appended to the value builder before this function is called will not be members of any list slot" or so



##########
cpp/src/arrow/c/bridge.cc:
##########
@@ -444,6 +444,10 @@ struct SchemaExporter {
 
   Status Visit(const LargeListType& type) { return SetFormat("+L"); }
 
+  Status Visit(const ListViewType& type) { return SetFormat("+lv"); }
+
+  Status Visit(const LargeListViewType& type) { return SetFormat("+Lv"); }

Review Comment:
   Looks like the vote has established
   ```suggestion
     Status Visit(const ListViewType& type) { return SetFormat("+vl"); }
   
     Status Visit(const LargeListViewType& type) { return SetFormat("+vL"); }
   ```



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/data.cc:
##########
@@ -233,9 +233,22 @@ BufferSpan OffsetsForScalar(uint8_t* scratch_space, offset_type value_size) {
   auto* offsets = reinterpret_cast<offset_type*>(scratch_space);
   offsets[0] = 0;
   offsets[1] = static_cast<offset_type>(value_size);
+  static_assert(2 * sizeof(offset_type) <= 16);
   return {scratch_space, sizeof(offset_type) * 2};
 }
 
+template <typename offset_type>
+std::pair<BufferSpan, BufferSpan> OffsetsAndSizesForScalar(uint8_t* scratch_space,
+                                                           offset_type value_size) {
+  auto* offsets = scratch_space;
+  auto* sizes = scratch_space + sizeof(offset_type);
+  reinterpret_cast<offset_type*>(offsets)[0] = 0;
+  reinterpret_cast<offset_type*>(sizes)[0] = value_size;

Review Comment:
   Yes. I will do it now. These utilities were added long after I started this PR. 💀



##########
cpp/src/arrow/array/data.cc:
##########
@@ -233,9 +233,22 @@ BufferSpan OffsetsForScalar(uint8_t* scratch_space, offset_type value_size) {
   auto* offsets = reinterpret_cast<offset_type*>(scratch_space);
   offsets[0] = 0;
   offsets[1] = static_cast<offset_type>(value_size);
+  static_assert(2 * sizeof(offset_type) <= 16);
   return {scratch_space, sizeof(offset_type) * 2};
 }
 
+template <typename offset_type>
+std::pair<BufferSpan, BufferSpan> OffsetsAndSizesForScalar(uint8_t* scratch_space,
+                                                           offset_type value_size) {
+  auto* offsets = scratch_space;
+  auto* sizes = scratch_space + sizeof(offset_type);
+  reinterpret_cast<offset_type*>(offsets)[0] = 0;
+  reinterpret_cast<offset_type*>(sizes)[0] = value_size;

Review Comment:
   Yes. I will do it now. These utilities were added long after I started this 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -181,7 +199,14 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   /// This function does the bare minimum of validation of the offsets and
   /// input types, and will allocate a new offsets array if necessary (i.e. if
   /// the offsets contain any nulls). If the offsets do not have nulls, they
-  /// are assumed to be well-formed
+  /// are assumed to be well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the
+  /// offsets's null bitmap. But if a null_bitmap is provided, the offsets array
+  /// can't have nulls.
+  ///
+  /// If a null_bitmap is provided, the offsets array can't be a slice (i.e. an
+  /// array with offset() > 0).

Review Comment:
   It's an overlook. For some reason, my brain is having a really hard time with these 4 slightly similar docstrings.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {

Review Comment:
   Yeah, that's the style. Fixing.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/ipc/writer.cc:
##########
@@ -442,6 +502,37 @@ class RecordBatchSerializer {
       // Must also slice the values
       values = values->Slice(values_offset, values_length);
     }
+    --max_recursion_depth_;
+    RETURN_NOT_OK(VisitArray(*values));
+    ++max_recursion_depth_;
+    return Status::OK();
+  }
+
+  template <typename T>
+  enable_if_list_view<typename T::TypeClass, Status> Visit(const T& array) {
+    using offset_type = typename T::offset_type;
+
+    offset_type min_offset = 0;
+    offset_type max_end = 0;
+    {
+      std::shared_ptr<Buffer> value_offsets;
+      RETURN_NOT_OK(
+          GetZeroBasedListViewOffsets<T>(array, &value_offsets, &min_offset, &max_end));

Review Comment:
   As above, I'm following the same logic in `GetZeroBasedValueOffsets`. This is how it is since the begining [1]. I don't know why.
   
   [1] https://github.com/wesm/arrow/commit/df2220f350282925a454ed911eed6618e4d53969#diff-1b1d9dca9fdea7624e22f017b8762c4919edf57c2cf43c15d59b8a5e8e1b38a5R264



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/integration/json_internal.cc:
##########
@@ -1492,6 +1506,14 @@ class ArrayReader {
     return CreateList<T>(type_);
   }
 
+  Status Visit(const ListViewType& type) {
+    return Status::NotImplemented("list-view in JSON");
+  }
+
+  Status Visit(const LargeListViewType& type) {
+    return Status::NotImplemented("large list-view in JSON");
+  }

Review Comment:
   Yes, you can.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -160,16 +166,69 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+void PutListViewOffsets(const Buffer& src, offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const BufferVector& buffers,
+                                  const std::vector<Range>& value_ranges,
+                                  MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  const int64_t out_size_in_bytes = SumBufferSizesInBytes(buffers);
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(out_size_in_bytes, pool));
+  auto* out_data = (*out)->mutable_data_as<offset_type>();
+
+  int64_t num_child_values = 0;
+  int64_t elements_length = 0;
+  for (size_t i = 0; i < buffers.size(); ++i) {
+    const auto displacement =
+        static_cast<offset_type>(num_child_values - value_ranges[i].offset);
+    PutListViewOffsets(/*src=*/*buffers[i], static_cast<offset_type>(displacement),
+                       /*dst=*/out_data + elements_length);
+    elements_length += buffers[i]->size() / sizeof(offset_type);

Review Comment:
   Oh, I think this is very personal, but I prefer code that doesn't mutate base buffer pointers and instead always re-derives the target pointer from the base pointer and an integer offset.
   
   If you show me how you would leverage the returned pointer to simplify this loop I can totally change it though. I can't see it by myself.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }
+
+namespace internal {
+
+/// \brief Calculate the smallest continuous range of values used by the
+/// var-length list-like input (list, map and list-view types).
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return A pair of (offset, length) describing the range
+ARROW_EXPORT Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(
+    const ArraySpan& input);
+
+/// \brief Calculate the sum of the sizes of all valid lists or list-views
+///
+/// This is usally the same as the length of the RangeOfValuesUsed() range, but
+/// it can be:
+/// - Smaller: when the child array constains many values that are not
+/// referenced by the lists or list-views in the parent array
+/// - Greater: when the list-views share child array ranges
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return The sum of all list or list-view sizes
+ARROW_EXPORT Result<int64_t> SumOfLogicalListSizes(const ArraySpan& input);
+
+/// \brief Build a ListViewArray from a ListArray
+ARROW_EXPORT Result<std::shared_ptr<ListViewArray>> ListViewFromList(
+    const ListArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListViewArray from a LargeListArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListViewArray>> ListViewFromList(
+    const LargeListArray& source, MemoryPool* pool);
+
+/// \brief Build a ListArray from a ListViewArray
+ARROW_EXPORT Result<std::shared_ptr<ListArray>> ListFromListView(
+    const ListViewArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListArray from a LargeListViewArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListArray>> ListFromListView(
+    const LargeListViewArray& source, MemoryPool* pool);

Review Comment:
   I tried it initially, but it brings so many dependencies to `array_nested.cc` (e.g. `MakeBuilder`) so I think it's preferable to keep the list+list-view stuff apart from the core array definitions.
   
   Another option is having these here and calls from `array_nested.cc`. Is that OK? Or should I move all this code into `array_nested.cc`?



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -602,8 +691,11 @@ class ConcatenateImpl {
 }  // namespace
 
 Result<std::shared_ptr<Array>> Concatenate(const ArrayVector& arrays, MemoryPool* pool) {
-  if (arrays.size() == 0) {
-    return Status::Invalid("Must pass at least one array");
+  switch (arrays.size()) {
+    case 0:
+      return Status::Invalid("Must pass at least one array");
+    case 1:
+      return arrays[0];

Review Comment:
   I will undo this. This was done after the initial reviews on this PR. I was also optimizing concatenation of children values when they were a single array, but I will also get rid of that and add an explicit comment about the 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   > What is the point of exposing this API on ListBuilder if sizes endures costly validation but is then entirely ignored? It doesn't seem to me like a very good idea, as builders are expected to be efficient.
   
   `ListBuilder` still exposes `AppendValues(offsets, length, valid_bytes)` with no loss of efficiency. `AppendValues` with `sizes` is overridden here so that we fulfill the generic contract of building var-length lists (requiring offsets + sizes).
   
   The expensive validation exists to catch people making mistakes of passing invalid `sizes` through the generic interface. They can gain more efficiency by passing `nullptr` on the sizes or using the `ListViewBuilder` directly. 
   
   > And if we want some compatible vector-append, then I suggest to do the reverse:
   expose a AppendValues(const offset_type* offsets, int64_t length, const uint8_t* valid_bytes) and let ListViewBuilder compute the sizes.
   
   Doing that has its problems as well: I can't delegate to a version that takes explicit `sizes` in list-view builders without allocating a temporary buffer. But I will implement like that and have you decide which way is better.



-- 
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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,188 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  std::pair<ListViewValidationError, int64_t> DoFullyValidateOffsetsAndSizes(
+      int64_t offset_limit) {
+    const auto* validity = data.GetValues<uint8_t>(0, 0);
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    int64_t slot = 0;
+    if (validity) {
+      internal::BitBlockCounter counter(validity, data.offset, data.length);
+      internal::BitBlockCount block;
+      for (int64_t i = 0; i < data.length; i += block.length) {
+        block = counter.NextWord();
+        if (block.NoneSet()) {
+          continue;
+        }
+        const bool all_set = block.AllSet();
+        for (int j = 0; j < block.length; j++) {
+          slot = i + j;
+          const bool valid = all_set || bit_util::GetBit(validity, data.offset + slot);
+          if (valid) {
+            const auto size = sizes[slot];
+            if (size > 0) {
+              const auto offset = offsets[slot];
+              if (offset < 0 || offset > offset_limit) {
+                return {kOutOfBoundsOffset, slot};
+              }
+              if (offset + size > offset_limit) {

Review Comment:
   Good catch. I can rewrite this to `if (size > offset_limit - offset)` that has no under/overflow issues because `offset <= offset_limit` after the conditional above has been judged `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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,55 +713,188 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  enum ListViewValidationError {
+    kOk = 0,
+    kOutOfBoundsOffset = 1,
+    kOutOfBoundsSize = 2,
+  };

Review Comment:
   Done. Pushing 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] felipecrv commented on a diff in pull request #35345: GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   As I tried to rewrite this I remembered why I didn't go with the version that infers sizes from offsets (that's what I tried at first): it's not possible to know if the offsets being passed contain the last offset of the array, so the sizes can't really be inferred.
   
   This is why the `sizes` validation doesn't care about the `size[length - 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -40,37 +40,46 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// List builder
+// VarLengthListLikeBuilder
 
 template <typename TYPE>
-class BaseListBuilder : public ArrayBuilder {
+class ARROW_EXPORT VarLengthListLikeBuilder : public ArrayBuilder {

Review Comment:
   We don't necessarily need the `Base-` prefix IMO. The goal of this and related interfaces is to allow virtual dispatching of actions that apply to all the var-length list-like (excluding fixed-length, including map) arrays.
   
   ![image](https://github.com/apache/arrow/assets/207795/541d9b7e-f05e-4fa4-a65c-2f090a93c89d)
   



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   Well, it's probably too late for 14.0.0 regardless, since @raulcd has already got the release branch.


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,349 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {
+    for (int64_t i = 0; i < input.length; i++) {
+      MINIMIZE_MIN_VIEW_OFFSET(i);
+    }
+  } else {
+    SetBitRunReader reader(validity, input.offset, input.length);
+    while (true) {
+      const auto run = reader.NextRun();
+      if (run.length == 0) {
+        break;
+      }
+      for (int64_t i = run.position; i < run.position + run.length; ++i) {
+        MINIMIZE_MIN_VIEW_OFFSET(i);
+      }
+    }
+  }
+  return min_offset.value_or(0);

Review Comment:
   If all the list-views are null or empty, the `min_offset` option will be empty and I return `0` which is safe because the `MaxViewEnd` will also be 0 in this case.
   
   There is an opportunity of optimization here that I hadn't noticed before: I can return the option, and if it's empty I don't have to call `MaxViewEnd`.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/type_traits.cc:
##########
@@ -67,21 +67,23 @@ int RequiredValueAlignmentForBuffer(Type::type type_id, int buffer_index) {
     case Type::BINARY:  // Offsets may be cast to int32_t*
     case Type::DATE32:
     case Type::TIME32:
-    case Type::LIST:  // Offsets may be cast to int32_t*, data is in child array
-    case Type::MAP:   // This is a list array
+    case Type::LIST:       // Offsets may be cast to int32_t*, data is in child array
+    case Type::LIST_VIEW:  // Same as LIST

Review Comment:
   (I've checkout the largelist, and it has, so maybe note here is proper?)



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +241,183 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. The offset and length of the offsets and sizes arrays must
+  /// match and that will be checked, but their contents will be assumed to be
+  /// well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the
+  /// offsets's null bitmap. But if a null_bitmap is provided, the offsets array
+  /// can't have nulls.
+  ///
+  /// If a null_bitmap is provided, the offsets array can't be a slice (i.e. an
+  /// array with offset() > 0).

Review Comment:
   No. I had the right comment in `LargeListView::FromArrays`. Copying it here and force pushing.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   When I started this PR, I had everything completely separate and then I saw myself writing a lot of tedious repetitive code. Then I spent quite some time refactoring everything from the beginning to leverage the commonalities between lists and list-views. I'm aware the builder implementation is not so clear, but the complexity of implementation pays itself by the ease of writing general code dealing with lists. This was proven in the unit tests code and will be even more important when we start talking about compute kernels dealing with lists.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +241,183 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. The offset and length of the offsets and sizes arrays must
+  /// match and that will be checked, but their contents will be assumed to be
+  /// well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the
+  /// offsets's null bitmap. But if a null_bitmap is provided, the offsets array
+  /// can't have nulls.
+  ///
+  /// If a null_bitmap is provided, the offsets array can't be a slice (i.e. an
+  /// array with offset() > 0).

Review Comment:
   But the sizes can?



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -181,7 +199,14 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   /// This function does the bare minimum of validation of the offsets and
   /// input types, and will allocate a new offsets array if necessary (i.e. if
   /// the offsets contain any nulls). If the offsets do not have nulls, they
-  /// are assumed to be well-formed
+  /// are assumed to be well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the
+  /// offsets's null bitmap. But if a null_bitmap is provided, the offsets array
+  /// can't have nulls.
+  ///
+  /// If a null_bitmap is provided, the offsets array can't be a slice (i.e. an
+  /// array with offset() > 0).

Review Comment:
   It's weird that this docstring snippet is different from the one in `ListArray` below. Is it an overlook?



##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -160,16 +166,69 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+void PutListViewOffsets(const Buffer& src, offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const BufferVector& buffers,
+                                  const std::vector<Range>& value_ranges,
+                                  MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  const int64_t out_size_in_bytes = SumBufferSizesInBytes(buffers);
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(out_size_in_bytes, pool));
+  auto* out_data = (*out)->mutable_data_as<offset_type>();
+
+  int64_t num_child_values = 0;
+  int64_t elements_length = 0;
+  for (size_t i = 0; i < buffers.size(); ++i) {
+    const auto displacement =
+        static_cast<offset_type>(num_child_values - value_ranges[i].offset);
+    PutListViewOffsets(/*src=*/*buffers[i], static_cast<offset_type>(displacement),
+                       /*dst=*/out_data + elements_length);
+    elements_length += buffers[i]->size() / sizeof(offset_type);

Review Comment:
   Perhaps `PutListViewOffsets` should return a pointer after the last written offset? (just a suggestion though)



##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -187,33 +189,89 @@ TEST_F(ConcatenateTest, FixedSizeListType) {
   });
 }
 
-TEST_F(ConcatenateTest, ListType) {
-  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+template <typename ListType>
+struct ListConcatenationChecker {
+  using offset_type = typename ListType::offset_type;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+  using ListArrayType = typename TypeTraits<ListType>::ArrayType;
+
+  template <typename Self>
+  static void Check(Self& self, int32_t size, double null_probability,
+                    std::shared_ptr<Array>* out) {
     auto values_size = size * 4;
-    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
-    auto offsets_vector = this->Offsets<int32_t>(values_size, size);
+    auto values =
+        self.template GeneratePrimitive<Int8Type>(values_size, null_probability);
+    auto offsets_vector = self.template Offsets<offset_type>(values_size, size);
     // Ensure first and last offsets encompass the whole values array
     offsets_vector.front() = 0;
-    offsets_vector.back() = static_cast<int32_t>(values_size);
+    offsets_vector.back() = static_cast<offset_type>(values_size);
     std::shared_ptr<Array> offsets;
-    ArrayFromVector<Int32Type>(offsets_vector, &offsets);
-    ASSERT_OK_AND_ASSIGN(*out, ListArray::FromArrays(*offsets, *values));
+    ArrayFromVector<OffsetArrowType>(offsets_vector, &offsets);
+    ASSERT_OK_AND_ASSIGN(*out, ListArrayType::FromArrays(*offsets, *values));
     ASSERT_OK((**out).ValidateFull());
+  }
+};
+
+TEST_F(ConcatenateTest, ListType) {
+  Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
+    ListConcatenationChecker<ListType>::Check(*this, size, null_probability, out);
   });
 }
 
 TEST_F(ConcatenateTest, LargeListType) {
   Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
-    auto values_size = size * 4;
-    auto values = this->GeneratePrimitive<Int8Type>(values_size, null_probability);
-    auto offsets_vector = this->Offsets<int64_t>(values_size, size);
-    // Ensure first and last offsets encompass the whole values array
-    offsets_vector.front() = 0;
-    offsets_vector.back() = static_cast<int64_t>(values_size);
+    ListConcatenationChecker<LargeListType>::Check(*this, size, null_probability, out);
+  });
+}
+
+template <typename ListViewType>
+struct ListViewConcatenationChecker {
+  using offset_type = typename ListViewType::offset_type;
+  using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+  using ListViewArrayType = typename TypeTraits<ListViewType>::ArrayType;
+
+  template <typename Self>
+  static void Check(Self& self, int32_t num_list_views, double null_probability,
+                    std::shared_ptr<Array>* out) {
+    auto values_size = 4 * num_list_views;
+    auto values =
+        self.template GeneratePrimitive<Int8Type>(values_size, null_probability);
+
     std::shared_ptr<Array> offsets;
-    ArrayFromVector<Int64Type>(offsets_vector, &offsets);
-    ASSERT_OK_AND_ASSIGN(*out, LargeListArray::FromArrays(*offsets, *values));
+    auto offsets_vector = self.template Offsets<offset_type>(values_size, num_list_views);
+    offsets_vector.front() = 0;

Review Comment:
   Do we want to enforce this? The test should be more interesting is the encompassed value range does not start at offset 0...



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -216,6 +241,183 @@ class ARROW_EXPORT LargeListArray : public BaseListArray<LargeListType> {
   void SetData(const std::shared_ptr<ArrayData>& data);
 };
 
+// ----------------------------------------------------------------------
+// ListViewArray / LargeListViewArray
+
+template <typename TYPE>
+class BaseListViewArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_view_type() const { return this->var_length_list_like_type(); }
+
+  /// Note that this buffer does not account for any slice offset or length.
+  const std::shared_ptr<Buffer>& value_sizes() const { return this->data_->buffers[2]; }
+
+  /// Return pointer to raw value offsets accounting for any slice offset
+  const offset_type* raw_value_sizes() const {
+    return raw_value_sizes_ + this->data_->offset;
+  }
+
+  offset_type value_length(int64_t i) const final {
+    return this->raw_value_sizes_[i + this->data_->offset];
+  }
+
+ protected:
+  const offset_type* raw_value_sizes_ = NULLPTR;
+};
+
+/// \brief Concrete Array class for list-view data
+class ARROW_EXPORT ListViewArray : public BaseListViewArray<ListViewType> {
+ public:
+  explicit ListViewArray(std::shared_ptr<ArrayData> data);
+
+  ListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                std::shared_ptr<Buffer> value_offsets,
+                std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct ListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct a ListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the child values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. The offset and length of the offsets and sizes arrays must
+  /// match and that will be checked, but their contents will be assumed to be
+  /// well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the
+  /// offsets's null bitmap. But if a null_bitmap is provided, the offsets array
+  /// can't have nulls.
+  ///
+  /// If a null_bitmap is provided, the offsets array can't be a slice (i.e. an
+  /// array with offset() > 0).
+  ///
+  /// \param[in] offsets An array of int32 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int32 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<ListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the list-views in this array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> offsets() const;
+
+  /// \brief Return list-view sizes as an Int32Array
+  ///
+  /// The returned array will not have a validity bitmap, so you cannot expect
+  /// to pass it to ListArray::FromArrays() and get back the same list array
+  /// if the original one has nulls.
+  std::shared_ptr<Array> sizes() const;
+
+ protected:
+  // This constructor defers SetData to a derived array class
+  ListViewArray() = default;
+
+  void SetData(const std::shared_ptr<ArrayData>& data);
+};
+
+/// \brief Concrete Array class for large list-view data (with 64-bit offsets
+/// and sizes)
+class ARROW_EXPORT LargeListViewArray : public BaseListViewArray<LargeListViewType> {
+ public:
+  explicit LargeListViewArray(std::shared_ptr<ArrayData> data);
+
+  LargeListViewArray(std::shared_ptr<DataType> type, int64_t length,
+                     std::shared_ptr<Buffer> value_offsets,
+                     std::shared_ptr<Buffer> value_sizes, std::shared_ptr<Array> values,
+                     std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+                     int64_t null_count = kUnknownNullCount, int64_t offset = 0);
+
+  /// \brief Construct LargeListViewArray from array of offsets, sizes, and child
+  /// value array
+  ///
+  /// Construct an LargeListViewArray using buffers from offsets and sizes arrays
+  /// that project views into the values array.
+  ///
+  /// This function does the bare minimum of validation of the offsets/sizes and
+  /// input types. The offset and length of the offsets and sizes arrays must
+  /// match and that will be checked, but their contents will be assumed to be
+  /// well-formed.
+  ///
+  /// If a null_bitmap is not provided, the nulls will be inferred from the offsets' or
+  /// sizes' null bitmap. Only one of these two is allowed to have a null bitmap. But if a
+  /// null_bitmap is provided, the offsets array and the sizes array can't have nulls.
+  ///
+  /// And when a null_bitmap is provided, neither the offsets or sizes array can be a
+  /// slice (i.e. an array with offset() > 0).
+  ///
+  /// \param[in] offsets An array of int64 offsets into the values array. NULL values are
+  /// supported if the corresponding values in sizes is NULL or 0.
+  /// \param[in] sizes An array containing the int64 sizes of every view. NULL values are
+  /// taken to represent a NULL list-view in the array being created.
+  /// \param[in] values Array containing list values
+  /// \param[in] pool MemoryPool
+  /// \param[in] null_bitmap Optional validity bitmap
+  /// \param[in] null_count Optional null count in null_bitmap
+  static Result<std::shared_ptr<LargeListViewArray>> FromArrays(
+      const Array& offsets, const Array& sizes, const Array& values,
+      MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  static Result<std::shared_ptr<LargeListViewArray>> FromArrays(
+      std::shared_ptr<DataType> type, const Array& offsets, const Array& sizes,
+      const Array& values, MemoryPool* pool = default_memory_pool(),
+      std::shared_ptr<Buffer> null_bitmap = NULLPTR,
+      int64_t null_count = kUnknownNullCount);
+
+  /// \brief Return an Array that is a concatenation of the large list-views in this
+  /// array.
+  ///
+  /// Note that it's different from `values()` in that it takes into
+  /// consideration this array's offsets (which can be in any order)
+  /// and sizes. Nulls are skipped.
+  Result<std::shared_ptr<Array>> Flatten(
+      MemoryPool* memory_pool = default_memory_pool()) const;
+
+  /// \brief Return list-view offsets as an Int64Array

Review Comment:
   Would recommend adding the same docstring description as in `ListViewArray` above.



##########
cpp/src/arrow/array/array_nested.h:
##########
@@ -105,6 +103,23 @@ class BaseListArray : public Array {
   const offset_type* raw_value_offsets_ = NULLPTR;
 };
 
+// ----------------------------------------------------------------------
+// ListArray / LargeListArray
+
+template <typename TYPE>
+class BaseListArray : public VarLengthListLikeArray<TYPE> {
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename TYPE::offset_type;
+
+  const TypeClass* list_type() const { return this->var_length_list_like_type(); }
+
+  offset_type value_length(int64_t i) const final {

Review Comment:
   So, just to be sure, `final` will allow devirtualizing this method if the object is known to be a `BaseListArray` or subclass thereof?



##########
cpp/src/arrow/ipc/test_common.cc:
##########
@@ -189,6 +189,30 @@ Status MakeRandomListArray(const std::shared_ptr<Array>& child_array, int num_li
   return MakeListArray<ListType>(child_array, num_lists, include_nulls, pool, out);
 }
 
+Status MakeRandomListViewArray(const std::shared_ptr<Array>& child_array, int num_lists,
+                               bool include_nulls, MemoryPool* pool,
+                               std::shared_ptr<Array>* out) {
+  const auto seed = static_cast<uint32_t>(child_array->length());
+  random::RandomArrayGenerator rand(seed);
+
+  const double null_probability = include_nulls ? 0.5 : 0.0;
+  *out = rand.ListView(*child_array, num_lists, null_probability, false, 0.9,
+                       kDefaultBufferAlignment, pool);
+  return Status::OK();
+}
+
+Status MakeRandomLargeListViewArray(const std::shared_ptr<Array>& child_array,
+                                    int num_lists, bool include_nulls, MemoryPool* pool,
+                                    std::shared_ptr<Array>* out) {
+  const auto seed = static_cast<uint32_t>(child_array->length());
+  random::RandomArrayGenerator rand(seed);
+
+  const double null_probability = include_nulls ? 0.5 : 0.0;
+  *out = rand.LargeListView(*child_array, num_lists, null_probability, false, 0.9,

Review Comment:
   Same here.



##########
cpp/src/arrow/ipc/test_common.cc:
##########
@@ -189,6 +189,30 @@ Status MakeRandomListArray(const std::shared_ptr<Array>& child_array, int num_li
   return MakeListArray<ListType>(child_array, num_lists, include_nulls, pool, out);
 }
 
+Status MakeRandomListViewArray(const std::shared_ptr<Array>& child_array, int num_lists,
+                               bool include_nulls, MemoryPool* pool,
+                               std::shared_ptr<Array>* out) {
+  const auto seed = static_cast<uint32_t>(child_array->length());
+  random::RandomArrayGenerator rand(seed);
+
+  const double null_probability = include_nulls ? 0.5 : 0.0;
+  *out = rand.ListView(*child_array, num_lists, null_probability, false, 0.9,

Review Comment:
   Can you mention argument names when they would otherwise be difficult to make out?
   ```suggestion
     *out = rand.ListView(*child_array, num_lists, null_probability, /*xxx=*/ false, /*yyy=*/ 0.9,
   ```



##########
cpp/src/arrow/ipc/writer.cc:
##########
@@ -442,6 +502,37 @@ class RecordBatchSerializer {
       // Must also slice the values
       values = values->Slice(values_offset, values_length);
     }
+    --max_recursion_depth_;
+    RETURN_NOT_OK(VisitArray(*values));
+    ++max_recursion_depth_;
+    return Status::OK();
+  }
+
+  template <typename T>
+  enable_if_list_view<typename T::TypeClass, Status> Visit(const T& array) {
+    using offset_type = typename T::offset_type;
+
+    offset_type min_offset = 0;
+    offset_type max_end = 0;
+    {
+      std::shared_ptr<Buffer> value_offsets;
+      RETURN_NOT_OK(
+          GetZeroBasedListViewOffsets<T>(array, &value_offsets, &min_offset, &max_end));

Review Comment:
   Does it mean we're attempting to compact the values array before sending it? This doesn't seem required, is it?



##########
cpp/src/arrow/testing/random.cc:
##########
@@ -811,6 +1021,12 @@ std::shared_ptr<Array> RandomArrayGenerator::ArrayOf(const Field& field, int64_t
     return *ARRAY_TYPE::FromArrays(field.type(), *offsets, *values);                 \
   }
 
+#define GENERATE_LIST_VIEW_CASE(ARRAY_TYPE)                                           \

Review Comment:
   Can you `#undef` it afterwards?



##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));
+    if (ARROW_PREDICT_TRUE(i != j)) {
+      // Swap validity bits
+      if (validity) {
+        const bool valid_i = bit_util::GetBit(validity, data.offset + i);
+        const bool valid_j = bit_util::GetBit(validity, data.offset + i);
+        if (valid_i != valid_j) {
+          bit_util::SetBitTo(validity, data.offset + i, valid_j);
+          bit_util::SetBitTo(validity, data.offset + j, valid_i);
+        }
+      }
+      // Swap offsets and sizes
+      std::swap(offsets[i], offsets[j]);
+      std::swap(sizes[i], sizes[j]);
+    }
+  }
+}
+
+/// \brief Generate the list-view offsets based on a random buffer of sizes.
+///
+/// The sizes buffer is an input of this function, but when force_empty_nulls is true,
+/// some values on the sizes buffer can be set to 0.
+///
+/// When sparsity is 0.0, the list-view spans are perfectly packed one after the
+/// other. If sparsity is greater than 0.0, the list-view spans are set apart
+/// from each other in proportion to the sparsity value and size of each
+/// list-view. A negative sparsity means each list-view shares a fraction of the
+/// values used by the previous list-view.
+///
+/// For instance, a sparsity of -1.0 means the values array will only need enough values
+/// for the largest list-view with all the other list-views spanning some of these same
+/// values.
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] mutable_sizes_array The array of sizes to use
+/// \param[in] force_empty_nulls Whether to force null list-view sizes to be 0
+/// \param[in] zero_undefined_offsets Whether to zero the offsets of list-views that have
+/// 0 set as the size
+/// \param[in] sparsity The sparsity of the generated list-view offsets
+/// \param[out] out_max_view_end The maximum value of the end of a list-view
+template <typename OffsetArrayType, typename offset_type>
+std::shared_ptr<Array> ViewOffsetsFromLengthsArray(
+    SeedType seed, OffsetArrayType& mutable_sizes_array, bool force_empty_nulls,
+    bool zero_undefined_offsets, double sparsity, int64_t* out_max_view_end,
+    int64_t alignment, MemoryPool* memory_pool) {
+  using TypeClass = typename OffsetArrayType::TypeClass;
+
+  auto* sizes = mutable_sizes_array.data()->template GetMutableValues<offset_type>(1);
+
+  BufferVector buffers{2};
+  buffers[0] = NULLPTR;  // sizes can have nulls, offsets don't have to
+  buffers[1] = *AllocateBuffer(sizeof(offset_type) * mutable_sizes_array.length(),
+                               alignment, memory_pool);
+  auto offsets = buffers[1]->mutable_data_as<offset_type>();
+
+  double offset_base = 0.0;
+  offset_type max_view_end = 0;
+  for (int64_t i = 0; i < mutable_sizes_array.length(); ++i) {
+    const auto offset = static_cast<offset_type>(std::llround(offset_base));
+    if (mutable_sizes_array.IsNull(i)) {
+      if (force_empty_nulls) {
+        sizes[i] = 0;
+      }
+      offsets[i] = zero_undefined_offsets ? 0 : offset;
+    } else {
+      if (sizes[i] == 0) {
+        offsets[i] = zero_undefined_offsets ? 0 : offset;
+      } else {
+        offsets[i] = offset;
+        DCHECK_LT(offset, std::numeric_limits<offset_type>::max() - sizes[i]);
+        offset_base = std::max(0.0, offset_base + (sparsity * sizes[i]));
+      }
+    }
+    max_view_end = std::max(max_view_end, offsets[i] + sizes[i]);
+  }
+  *out_max_view_end = max_view_end;
+
+  auto array_data =
+      ArrayData::Make(TypeTraits<TypeClass>::type_singleton(),
+                      mutable_sizes_array.length(), std::move(buffers), /*null_count=*/0);
+  return std::make_shared<OffsetArrayType>(std::move(array_data));
+}
+
+template <typename ArrayType, typename RAG>
+Result<std::shared_ptr<Array>> ArrayOfListView(RAG& self, const Field& field,

Review Comment:
   Hmm... why do `RandomListView` and `ArrayOfListView` use two different approaches? This is more code to maintain and understand.
   
   (I'm not sure which approach is better, but ideally we would decide on a common one)



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -40,37 +40,46 @@ namespace arrow {
 /// @{
 
 // ----------------------------------------------------------------------
-// List builder
+// VarLengthListLikeBuilder
 
 template <typename TYPE>
-class BaseListBuilder : public ArrayBuilder {
+class ARROW_EXPORT VarLengthListLikeBuilder : public ArrayBuilder {

Review Comment:
   Fair enough, let's not bikeshed this :-)



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {
+    for (int64_t i = 0; i < input.length; i++) {
+      MINIMIZE_MIN_VIEW_OFFSET(i);
+    }
+  } else {
+    SetBitRunReader reader(validity, input.offset, input.length);
+    while (true) {
+      const auto run = reader.NextRun();
+      if (run.length == 0) {
+        break;
+      }
+      for (int64_t i = run.position; i < run.position + run.length; ++i) {
+        MINIMIZE_MIN_VIEW_OFFSET(i);
+      }
+    }
+  }
+  return min_offset;
+
+#undef MINIMIZE_MIN_VIEW_OFFSET
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  const auto values_length = input.child_data[0].length;
+
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Early-exit: 64-bit overflow detected. This is not possible on a valid list-view,
+  // but we return the maximum possible value to avoid undefined behavior.

Review Comment:
   Unless there's a specific condition where this can be called on invalid data (for example when reading IPC data), we don't care that UB would happen if the list-view is invalid.



##########
cpp/src/arrow/array/util.cc:
##########
@@ -383,6 +393,12 @@ class NullArrayFactory {
       return Status::OK();
     }
 
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T&) {
+      buffer_length_ = length_ * sizeof(typename T::offset_type);

Review Comment:
   Also remember to recurse on the child array as in the var_size_list case above (though it seems it should call `GetBufferLength` with length 0, not `length_`... @bkietz WDYT ?).



##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   Hmm, I'm coming back to this discussion and I'm struggling to understand this again.
   
   This seems to hint that List builders and ListView builders are different enough that they should probably be different beasts entirely instead of trying to maximize API and code sharing. What do you think? 
   



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -797,57 +811,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > ",
+                             offset_limit);
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    for (int64_t i = 0; i < data.length; ++i) {
+      const auto size = sizes[i];
+      if (size >= 0) {
+        const auto offset = offsets[i];
+        if (offset < 0 || offset > offset_limit) {
+          return OutOfBoundsListViewOffset<offset_type>(i, offset_limit);
+        }
+        if (size > offset_limit - offset) {
+          return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+        }
+      } else {
+        return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
-    if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+    const bool non_empty = data.length > 0;
+    if constexpr (is_list_view) {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).

Review Comment:
   Remove this comment?



##########
cpp/src/arrow/array/util.cc:
##########
@@ -383,6 +393,12 @@ class NullArrayFactory {
       return Status::OK();
     }
 
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T&) {
+      buffer_length_ = length_ * sizeof(typename T::offset_type);

Review Comment:
   Shouldn't you call `MaxOf` instead of setting `buffer_length_` directly?



##########
cpp/src/arrow/ipc/writer.cc:
##########
@@ -350,6 +350,67 @@ class RecordBatchSerializer {
     return Status::OK();
   }
 
+  template <typename ArrayType, typename offset_type = typename ArrayType::offset_type>
+  Status GetZeroBasedListViewOffsets(const ArrayType& array,
+                                     std::shared_ptr<Buffer>* out_value_offsets,
+                                     offset_type* out_min_offset,
+                                     offset_type* out_max_end) {
+    auto offsets = array.value_offsets();
+    auto sizes = array.value_sizes();
+
+    const int64_t required_bytes = sizeof(offset_type) * array.length();
+    if (array.offset() != 0) {
+      // If we have a non-zero offset, it's likely that the smallest offset is
+      // not zero. We must a) create a new offsets array with shifted offsets and
+      // b) slice the values array accordingly.
+
+      ARROW_ASSIGN_OR_RAISE(auto shifted_offsets,
+                            AllocateBuffer(required_bytes, options_.memory_pool));
+      offset_type min_offset = 0;
+      offset_type max_end = 0;
+      if (array.length() > 0) {
+        min_offset = std::numeric_limits<offset_type>::max();
+        for (int i = 0; i < array.length(); ++i) {
+          min_offset = std::min(min_offset, array.value_offset(i));
+          max_end = std::max(max_end, array.value_offset(i) + array.value_length(i));
+        }
+      }
+
+      auto* dest_offsets = shifted_offsets->mutable_data_as<offset_type>();
+
+      for (int i = 0; i < array.length(); ++i) {
+        dest_offsets[i] = array.value_offset(i) - min_offset;
+      }
+      *out_min_offset = min_offset;
+      *out_max_end = max_end;
+      offsets = std::move(shifted_offsets);
+    } else {
+      // ARROW-6046: Slice offsets to used extent, in case we have a truncated
+      // slice
+      if (offsets != nullptr && offsets->size() > required_bytes) {
+        offsets = SliceBuffer(offsets, 0, required_bytes);
+      }
+      *out_min_offset = 0;
+      *out_max_end = static_cast<offset_type>(array.values()->length());
+    }
+    *out_value_offsets = std::move(offsets);
+    return Status::OK();
+  }
+
+  template <typename ArrayType, typename offset_type = typename ArrayType::offset_type>
+  Status GetListViewSizes(const ArrayType& array,
+                          std::shared_ptr<Buffer>* out_value_sizes) {
+    const int64_t required_bytes = sizeof(offset_type) * array.length();
+    auto sizes = array.value_sizes();
+    if (sizes != nullptr && (array.offset() != 0 || sizes->size() > required_bytes)) {

Review Comment:
   Why would `sizes` be null? Also, the second condition seems a bit pedantic, since slicing is cheap.



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -797,57 +811,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    auto prev_offset = offsets[0];
+    if (prev_offset < 0) {
+      return Status::Invalid("Offset invariant failure: array starts at negative offset ",
+                             prev_offset);
+    }
+    for (int64_t i = 1; i <= data.length; ++i) {
+      const auto current_offset = offsets[i];
+      if (current_offset < prev_offset) {
+        return Status::Invalid("Offset invariant failure: non-monotonic offset at slot ",
+                               i, ": ", current_offset, " < ", prev_offset);
+      }
+      if (current_offset > offset_limit) {
+        return Status::Invalid("Offset invariant failure: offset for slot ", i,
+                               " out of bounds: ", current_offset, " > ", offset_limit);
+      }
+      prev_offset = current_offset;
+    }
+    return Status::OK();
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewOffset(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto offset = offsets[slot];
+    return Status::Invalid("Offset invariant failure: offset for slot ", slot,
+                           " out of bounds. Expected ", offset,
+                           " to be at least 0 and less than ", offset_limit);
+  }
+
+  template <typename offset_type>
+  Status OutOfBoundsListViewSize(int64_t slot, int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+    const auto size = sizes[slot];
+    if (size < 0) {
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", size, " < 0");
+    } else {
+      const auto offset = offsets[slot];
+      return Status::Invalid("Offset invariant failure: size for slot ", slot,
+                             " out of bounds: ", offset, " + ", size, " > ",
+                             offset_limit);
+    }
+  }
+
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsetsAndSizes(int64_t offset_limit) {
+    const auto* offsets = data.GetValues<offset_type>(1);
+    const auto* sizes = data.GetValues<offset_type>(2);
+
+    for (int64_t i = 0; i < data.length; ++i) {
+      const auto size = sizes[i];
+      if (size >= 0) {
+        const auto offset = offsets[i];
+        if (offset < 0 || offset > offset_limit) {
+          return OutOfBoundsListViewOffset<offset_type>(i, offset_limit);
+        }
+        if (size > offset_limit - offset) {
+          return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+        }
+      } else {
+        return OutOfBoundsListViewSize<offset_type>(i, offset_limit);
+      }
+    }
+
+    return Status::OK();
+  }
+
   template <typename TypeClass>
-  Status ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+  Status ValidateOffsetsAndMaybeSizes(const TypeClass&, int64_t offset_limit) {
     using offset_type = typename TypeClass::offset_type;
+    constexpr bool is_list_view = is_list_view_type<TypeClass>::value;
 
-    if (!IsBufferValid(1)) {
-      // For length 0, an empty offsets buffer seems accepted as a special case
-      // (ARROW-544)
-      if (data.length > 0) {
-        return Status::Invalid("Non-empty array but offsets are null");
+    const bool non_empty = data.length > 0;
+    if constexpr (is_list_view) {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return Status::Invalid("offsets buffer is null");
+      }
+      if (!IsBufferValid(2)) {
+        return Status::Invalid("sizes buffer is null");
+      }
+    } else {
+      if (!IsBufferValid(1)) {
+        // For length 0, an empty offsets buffer is accepted (ARROW-544).
+        return non_empty ? Status::Invalid("Non-empty array but offsets are null")
+                         : Status::OK();
       }
-      return Status::OK();
     }
 
-    // An empty list array can have 0 offsets
     const auto offsets_byte_size = data.buffers[1]->size();
     const auto required_offsets = ((data.length > 0) || (offsets_byte_size > 0))
-                                      ? data.length + data.offset + 1
+                                      ? data.length + data.offset + (is_list_view ? 0 : 1)
                                       : 0;
     if (offsets_byte_size / static_cast<int32_t>(sizeof(offset_type)) <
         required_offsets) {
       return Status::Invalid("Offsets buffer size (bytes): ", offsets_byte_size,
                              " isn't large enough for length: ", data.length,
                              " and offset: ", data.offset);
     }
+    if constexpr (is_list_view) {
+      const auto required_sizes = data.length + data.offset;
+      const auto sizes_bytes_size = data.buffers[2]->size();
+      if (sizes_bytes_size / static_cast<int32_t>(sizeof(offset_type)) < required_sizes) {
+        return Status::Invalid("Sizes buffer size (bytes): ", sizes_bytes_size,
+                               " isn't large enough for length: ", data.length,
+                               " and offset: ", data.offset);
+      }
+    }
 
     if (full_validation && required_offsets > 0) {
-      // Validate all offset values
-      const offset_type* offsets = data.GetValues<offset_type>(1);
-
-      auto prev_offset = offsets[0];
-      if (prev_offset < 0) {
-        return Status::Invalid(
-            "Offset invariant failure: array starts at negative offset ", prev_offset);
-      }
-      for (int64_t i = 1; i <= data.length; ++i) {
-        const auto current_offset = offsets[i];
-        if (current_offset < prev_offset) {
-          return Status::Invalid(
-              "Offset invariant failure: non-monotonic offset at slot ", i, ": ",
-              current_offset, " < ", prev_offset);
-        }
-        if (current_offset > offset_limit) {
-          return Status::Invalid("Offset invariant failure: offset for slot ", i,
-                                 " out of bounds: ", current_offset, " > ", offset_limit);
-        }
-        prev_offset = current_offset;
+      if constexpr (is_list_view) {
+        return FullyValidateOffsetsAndSizes<offset_type>(offset_limit);
+      } else {
+        return FullyValidateOffsets<offset_type>(offset_limit);
       }
     }
     return Status::OK();
   }
 
+ public:
+  template <typename TypeClass>
+  enable_if_list_view<TypeClass, Status> ValidateOffsetsAndSizes(const TypeClass& type,
+                                                                 int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }
+
+  template <typename TypeClass>
+  std::enable_if_t<is_var_length_list_type<TypeClass>::value ||
+                       is_base_binary_like(TypeClass::type_id),
+                   Status>
+  ValidateOffsets(const TypeClass& type, int64_t offset_limit) {
+    return ValidateOffsetsAndMaybeSizes<TypeClass>(type, offset_limit);
+  }

Review Comment:
   I'm not sure I understand the point of these two indirections.



##########
cpp/src/arrow/array/validate.cc:
##########
@@ -23,7 +23,7 @@
 #include "arrow/extension_type.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"
-#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_run_reader.h"

Review Comment:
   Was it a problem in the existing code?



##########
cpp/src/arrow/array/util.cc:
##########
@@ -853,6 +885,13 @@ class RepeatedArrayFactory {
     return builder.Finish(out);
   }
 
+  template <typename IntType>
+  Status CreateIntBuffer(IntType value, std::shared_ptr<Buffer>* out) {

Review Comment:
   Can return `Result<std::shared_ptr<Buffer>>` for more convenient usage.



##########
cpp/src/arrow/c/bridge_test.cc:
##########
@@ -1930,6 +2004,33 @@ TEST_F(TestSchemaImport, NestedList) {
   CheckImport(list(fixed_size_list(int8(), 3)));
 }
 
+TEST_F(TestSchemaImport, ListView) {
+  FillPrimitive(AddChild(), "c");
+  FillListLike("+vl");
+  CheckImport(list_view(int8()));
+
+  FillPrimitive(AddChild(), "s", "item", 0);
+  FillListLike("+vl");
+  CheckImport(list_view(field("item", int16(), /*nullable=*/false)));
+
+  // Large list-view
+  FillPrimitive(AddChild(), "s");
+  FillListLike("+vL");
+  CheckImport(large_list_view(int16()));
+}
+
+TEST_F(TestSchemaImport, NestedListView) {
+  FillPrimitive(AddChild(), "c");
+  FillListLike(AddChild(), "+vl");
+  FillListLike("+vL");
+  CheckImport(large_list_view(list_view(int8())));
+
+  FillPrimitive(AddChild(), "c");
+  FillListLike(AddChild(), "+w:3");
+  FillListLike("+vl");
+  CheckImport(list_view(fixed_size_list(int8(), 3)));
+}
+

Review Comment:
   Are we missing tests for array import and roundtrip?



##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }
+
+namespace internal {
+
+/// \brief Calculate the smallest continuous range of values used by the
+/// var-length list-like input (list, map and list-view types).
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return A pair of (offset, length) describing the range
+ARROW_EXPORT Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(
+    const ArraySpan& input);
+
+/// \brief Calculate the sum of the sizes of all valid lists or list-views
+///
+/// This is usally the same as the length of the RangeOfValuesUsed() range, but
+/// it can be:
+/// - Smaller: when the child array constains many values that are not
+/// referenced by the lists or list-views in the parent array
+/// - Greater: when the list-views share child array ranges
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return The sum of all list or list-view sizes
+ARROW_EXPORT Result<int64_t> SumOfLogicalListSizes(const ArraySpan& input);
+
+/// \brief Build a ListViewArray from a ListArray
+ARROW_EXPORT Result<std::shared_ptr<ListViewArray>> ListViewFromList(
+    const ListArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListViewArray from a LargeListArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListViewArray>> ListViewFromList(
+    const LargeListArray& source, MemoryPool* pool);
+
+/// \brief Build a ListArray from a ListViewArray
+ARROW_EXPORT Result<std::shared_ptr<ListArray>> ListFromListView(
+    const ListViewArray& source, MemoryPool* pool);
+
+/// \brief Build a LargeListArray from a LargeListViewArray
+ARROW_EXPORT Result<std::shared_ptr<LargeListArray>> ListFromListView(
+    const LargeListViewArray& source, MemoryPool* pool);

Review Comment:
   It would seem more logical to me to have these as static methods of `ListArray` and `ListViewArray`.



##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+
+/// \brief Get the child array holding the values from a List or ListView array
+inline const ArraySpan& ValuesArray(const ArraySpan& span) { return span.child_data[0]; }

Review Comment:
   This honestly looks like a pointless indirection to me, though I wouldn't strongly oppose it either.
   
   (also, should it be `constexpr`?)



##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {
+  DCHECK_EQ(data.type->id(), ListViewType::type_id);
+  using offset_type = typename ListViewType::offset_type;
+
+  auto* validity = data.GetMutableValues<uint8_t>(0, 0);
+  auto* offsets = data.GetMutableValues<offset_type>(1);
+  auto* sizes = data.GetMutableValues<offset_type>(2);
+
+  pcg32_fast rng(seed);
+  using UniformDist = std::uniform_int_distribution<int64_t>;
+  UniformDist dist;
+  for (int64_t i = data.length - 1; i > 0; --i) {
+    const auto j = dist(rng, UniformDist::param_type(0, i));

Review Comment:
   I didn't know this two-operand form :-o



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \

Review Comment:
   We should ideally keep macros trivial. Here, it can be replaced with a capturing lambda.



##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {

Review Comment:
   Can simply use `VisitSetBitRunsVoid` which automates this scaffolding:
   ```c++
     VisitSetBitRunsVoid(validity, input.offset, input.length(), [&](int64_t start, int64t_ length) { ... });
   ```



##########
cpp/src/arrow/testing/random.cc:
##########
@@ -608,6 +609,218 @@ std::shared_ptr<Array> OffsetsFromLengthsArray(OffsetArrayType* lengths,
       std::make_shared<typename OffsetArrayType::TypeClass>(), size, buffers, null_count);
   return std::make_shared<OffsetArrayType>(array_data);
 }
+
+// Helper for RandomArrayGenerator::ArrayOf: extract some C value from
+// a given metadata key.
+template <typename T, typename ArrowType = typename CTypeTraits<T>::ArrowType>
+enable_if_parameter_free<ArrowType, T> GetMetadata(const KeyValueMetadata* metadata,
+                                                   const std::string& key,
+                                                   T default_value) {
+  if (!metadata) return default_value;
+  const auto index = metadata->FindKey(key);
+  if (index < 0) return default_value;
+  const auto& value = metadata->value(index);
+  T output{};
+  if (!internal::ParseValue<ArrowType>(value.data(), value.length(), &output)) {
+    ABORT_NOT_OK(Status::Invalid("Could not parse ", key, " = ", value, " as ",
+                                 ArrowType::type_name()));
+  }
+  return output;
+}
+
+/// \brief Shuffle a list-view array in place using the Fisher–Yates algorithm [1].
+///
+/// [1] https://en.wikipedia.org/wiki/Fisher%E2%80%93Yates_shuffle#The_modern_algorithm
+///
+/// \param[in] seed The seed for the random number generator
+/// \param[in,out] data The array to shuffle
+template <typename ListViewType>
+void ShuffleListViewDataInPlace(SeedType seed, ArrayData& data) {

Review Comment:
   Ideally we would avoid mutable references.
   ```suggestion
   void ShuffleListViewDataInPlace(SeedType seed, ArrayData* 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/validate.cc:
##########
@@ -699,57 +713,147 @@ struct ValidateArrayImpl {
     return Status::OK();
   }
 
+ private:
+  /// \pre basic validation has already been performed
+  template <typename offset_type>
+  Status FullyValidateOffsets(int64_t offset_limit) {

Review Comment:
   Yes. The ListView function is `FullyValidateOffsetsAndValues` as they have to be validated together to guarantee `offset[i] + size[i] <= values.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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   @mapleFU I don't understand this comment https://github.com/apache/arrow/pull/35345#discussion_r1359476421 can you clarify?


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   Well I think build the List with size is ok. But can we add some comment for how to use that? Since that would be a bit tricky.


-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -115,8 +138,16 @@ class ARROW_EXPORT VarLengthListLikeBuilder : public ArrayBuilder {
     return Status::OK();
   }
 
+  /// \brief Append an empty list slot
+  ///
+  /// \post Another call to Append* or Finish should be made before appending to
+  /// the values builder to ensure list slot remains empty

Review Comment:
   Note that this was already an unwritten contract when there were only regular list builders.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate.cc:
##########
@@ -160,16 +168,144 @@ Status PutOffsets(const std::shared_ptr<Buffer>& src, Offset first_offset, Offse
 
   // Write offsets into dst, ensuring that the first offset written is
   // first_offset
-  auto adjustment = first_offset - src_begin[0];
+  auto displacement = first_offset - src_begin[0];
   // NOTE: Concatenate can be called during IPC reads to append delta dictionaries.
   // Avoid UB on non-validated input by doing the addition in the unsigned domain.
   // (the result can later be validated using Array::ValidateFull)
-  std::transform(src_begin, src_end, dst, [adjustment](Offset offset) {
-    return SafeSignedAdd(offset, adjustment);
+  std::transform(src_begin, src_end, dst, [displacement](Offset offset) {
+    return SafeSignedAdd(offset, displacement);
   });
   return Status::OK();
 }
 
+template <typename offset_type>
+Status PutListViewOffsets(const ArrayData& input, offset_type* sizes, const Buffer& src,
+                          offset_type displacement, offset_type* dst);
+
+// Concatenate buffers holding list-view offsets into a single buffer of offsets
+//
+// value_ranges contains the relevant ranges of values in the child array actually
+// referenced to by the views. Most commonly, these ranges will start from 0,
+// but when that is not the case, we need to adjust the displacement of offsets.
+// The concatenated child array does not contain values from the beginning
+// if they are not referenced to by any view.
+//
+// The child arrays and the sizes buffer are used to ensure we can trust the offsets in
+// offset_buffers to be within the valid range.
+//
+// This function also mutates sizes so that null list-view entries have size 0.
+//
+// \param[in] in The child arrays
+// \param[in,out] sizes The concatenated sizes buffer
+template <typename offset_type>
+Status ConcatenateListViewOffsets(const ArrayDataVector& in, offset_type* sizes,
+                                  const BufferVector& offset_buffers,
+                                  const std::vector<Range>& value_ranges,
+                                  MemoryPool* pool, std::shared_ptr<Buffer>* out) {
+  DCHECK_EQ(offset_buffers.size(), value_ranges.size());
+
+  // Allocate resulting offsets buffer and initialize it with zeros
+  const int64_t out_size_in_bytes = SumBufferSizesInBytes(offset_buffers);
+  ARROW_ASSIGN_OR_RAISE(*out, AllocateBuffer(out_size_in_bytes, pool));
+  memset((*out)->mutable_data(), 0, static_cast<size_t>((*out)->size()));
+
+  auto* out_offsets = (*out)->mutable_data_as<offset_type>();
+
+  int64_t num_child_values = 0;
+  int64_t elements_length = 0;
+  for (size_t i = 0; i < offset_buffers.size(); ++i) {
+    const auto displacement =
+        static_cast<offset_type>(num_child_values - value_ranges[i].offset);
+    RETURN_NOT_OK(PutListViewOffsets(*in[i], /*sizes=*/sizes + elements_length,
+                                     /*src=*/*offset_buffers[i], displacement,
+                                     /*dst=*/out_offsets + elements_length));
+    elements_length += offset_buffers[i]->size() / sizeof(offset_type);
+    num_child_values += value_ranges[i].length;
+    if (num_child_values > std::numeric_limits<offset_type>::max()) {
+      return Status::Invalid("offset overflow while concatenating arrays");
+    }
+  }
+  DCHECK_EQ(elements_length,
+            static_cast<int64_t>(out_size_in_bytes / sizeof(offset_type)));
+
+  return Status::OK();
+}
+
+template <typename offset_type>
+Status PutListViewOffsets(const ArrayData& input, offset_type* sizes, const Buffer& src,
+                          offset_type displacement, offset_type* dst) {
+  if (src.size() == 0) {
+    return Status::OK();
+  }
+  const auto& validity_buffer = input.buffers[0];
+  if (validity_buffer) {
+    // Ensure that it is safe to access all the bits in the validity bitmap of input.
+    RETURN_NOT_OK(internal::CheckSliceParams(/*size=*/8 * validity_buffer->size(),
+                                             input.offset, input.length, "buffer"));
+  }
+
+  const auto offsets = src.data_as<offset_type>();
+  DCHECK_EQ(static_cast<int64_t>(src.size() / sizeof(offset_type)), input.length);
+
+  auto visit_not_null = [&](int64_t position) {
+    if (sizes[position] > 0) {
+      // NOTE: Concatenate can be called during IPC reads to append delta
+      // dictionaries. Avoid UB on non-validated input by doing the addition in the
+      // unsigned domain. (the result can later be validated using
+      // Array::ValidateFull)
+      const auto displaced_offset = SafeSignedAdd(offsets[position], displacement);
+      // displaced_offset>=0 is guaranteed by RangeOfValuesUsed returning the
+      // smallest offset of valid and non-empty list-views.
+      DCHECK_GE(displaced_offset, 0);
+      dst[position] = displaced_offset;
+    } else {
+      // Do nothing to leave the dst[position] as 0.
+    }
+  };
+
+  const auto* validity = validity_buffer ? validity_buffer->data_as<uint8_t>() : nullptr;
+  internal::OptionalBitBlockCounter bit_counter(validity, input.offset, input.length);
+  int64_t position = 0;
+  while (position < input.length) {
+    internal::BitBlockCount block = bit_counter.NextBlock();
+    if (block.AllSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        if (sizes[position] > 0) {
+          // NOTE: Concatenate can be called during IPC reads to append delta
+          // dictionaries. Avoid UB on non-validated input by doing the addition in the
+          // unsigned domain. (the result can later be validated using
+          // Array::ValidateFull)
+          const auto displaced_offset = SafeSignedAdd(offsets[position], displacement);
+          // displaced_offset>=0 is guaranteed by RangeOfValuesUsed returning the
+          // smallest offset of valid and non-empty list-views.
+          DCHECK_GE(displaced_offset, 0);
+          dst[position] = displaced_offset;
+        } else {
+          // Do nothing to leave dst[position] as 0.
+        }

Review Comment:
   I might be misreading, but is it just the same as `visit_not_null(i)`?



##########
cpp/src/arrow/util/list_util.h:
##########
@@ -0,0 +1,55 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <utility>
+
+#include "arrow/array/data.h"
+#include "arrow/result.h"
+
+namespace arrow {
+namespace list_util {
+namespace internal {
+
+/// \brief Calculate the smallest continuous range of values used by the
+/// var-length list-like input (list, map and list-view types).
+///
+/// \param input The input array such that is_var_length_list_like(input.type)
+/// is true
+/// \return A pair of (offset, length) describing the range
+ARROW_EXPORT Result<std::pair<int64_t, int64_t>> RangeOfValuesUsed(
+    const ArraySpan& input);
+
+/// \brief Calculate the sum of the sizes of all valid lists or list-views
+///
+/// This is usally the same as the length of the RangeOfValuesUsed() range, but

Review Comment:
   ```suggestion
   /// This is usually the same as the length of the RangeOfValuesUsed() range, but
   ```



##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -69,33 +99,117 @@ class ConcatenateTest : public ::testing::Test {
     return slices;
   }
 
+  std::shared_ptr<Buffer> ValidityBitmap(int64_t size, double null_probability) {
+    return rag_.NullBitmap(size, null_probability, kDefaultBufferAlignment,
+                           default_memory_pool());
+  }
+
   template <typename PrimitiveType>
-  std::shared_ptr<Array> GeneratePrimitive(int64_t size, double null_probability) {
+  std::shared_ptr<Array> PrimitiveArray(int64_t size, double null_probability) {
     if (std::is_same<PrimitiveType, BooleanType>::value) {
-      return rng_.Boolean(size, 0.5, null_probability);
+      return rag_.Boolean(size, 0.5, null_probability);
     }
-    return rng_.Numeric<PrimitiveType, uint8_t>(size, 0, 127, null_probability);
+    return rag_.Numeric<PrimitiveType, uint8_t>(size, 0, 127, null_probability);
+  }
+
+  std::shared_ptr<Array> StringArray(int64_t size, double null_probability) {
+    return rag_.String(size, /*min_length =*/0, /*max_length =*/15, null_probability);
+  }
+
+  std::shared_ptr<Array> LargeStringArray(int64_t size, double null_probability) {
+    return rag_.LargeString(size, /*min_length =*/0, /*max_length =*/15,
+                            null_probability);
+  }
+
+  std::shared_ptr<Array> StringViewArray(int64_t size, double null_probability) {
+    return rag_.StringView(size, /*min_length =*/0, /*max_length =*/40, null_probability,
+                           /*max_buffer_length=*/200);
+  }
+
+  std::shared_ptr<Array> ArrayOf(std::shared_ptr<DataType> type, int64_t size,
+                                 double null_probability) {
+    return rag_.ArrayOf(std::move(type), size, null_probability);
+  }
+
+  // TODO(GH-38656): Use the random array generators from testing/random.h here
+
+  template <typename ListType,
+            typename ListArrayType = typename TypeTraits<ListType>::ArrayType>
+  Result<std::shared_ptr<ListArrayType>> ListArray(int32_t length,
+                                                   double null_probability) {
+    using offset_type = typename ListType::offset_type;
+    using OffsetArrowType = typename CTypeTraits<offset_type>::ArrowType;
+
+    auto values_size = length * 4;
+    auto values = PrimitiveArray<Int8Type>(values_size, null_probability);
+    auto offsets_vector = Offsets<offset_type>(values_size, length);
+    // Ensure first and last offsets encompass the whole values array
+    offsets_vector.front() = 0;

Review Comment:
   Thanks!



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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

   > @felipecrv We'll want to update https://github.com/apache/arrow/blob/main/docs/source/status.rst in a followup PR.
   
   I will be extremely glad to send that 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/util.cc:
##########
@@ -379,6 +389,15 @@ class NullArrayFactory {
     enable_if_var_size_list<T, Status> Visit(const T& type) {
       // values array may be empty, but there must be at least one offset of 0
       RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * (length_ + 1)));
+      // XXX(felipec): reviewers, is this correct?
+      RETURN_NOT_OK(MaxOf(GetBufferLength(type.value_type(), length_)));
+      return Status::OK();
+    }
+
+    template <typename T>
+    enable_if_list_view<T, Status> Visit(const T& type) {
+      RETURN_NOT_OK(MaxOf(sizeof(typename T::offset_type) * length_));
+      // XXX(felipec): reviewers, is this correct?

Review Comment:
   Removed the XXX comments and pushed 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


Re: [PR] GH-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/concatenate_test.cc:
##########
@@ -69,33 +99,117 @@ class ConcatenateTest : public ::testing::Test {
     return slices;
   }
 
+  std::shared_ptr<Buffer> ValidityBitmap(int64_t size, double null_probability) {
+    return rag_.NullBitmap(size, null_probability, kDefaultBufferAlignment,
+                           default_memory_pool());
+  }
+
   template <typename PrimitiveType>
-  std::shared_ptr<Array> GeneratePrimitive(int64_t size, double null_probability) {
+  std::shared_ptr<Array> PrimitiveArray(int64_t size, double null_probability) {
     if (std::is_same<PrimitiveType, BooleanType>::value) {
-      return rng_.Boolean(size, 0.5, null_probability);
+      return rag_.Boolean(size, 0.5, null_probability);
     }
-    return rng_.Numeric<PrimitiveType, uint8_t>(size, 0, 127, null_probability);
+    return rag_.Numeric<PrimitiveType, uint8_t>(size, 0, 127, null_probability);
+  }
+
+  std::shared_ptr<Array> StringArray(int64_t size, double null_probability) {
+    return rag_.String(size, /*min_length =*/0, /*max_length =*/15, null_probability);
+  }
+
+  std::shared_ptr<Array> LargeStringArray(int64_t size, double null_probability) {
+    return rag_.LargeString(size, /*min_length =*/0, /*max_length =*/15,
+                            null_probability);
+  }
+
+  std::shared_ptr<Array> StringViewArray(int64_t size, double null_probability) {
+    return rag_.StringView(size, /*min_length =*/0, /*max_length =*/40, null_probability,
+                           /*max_buffer_length=*/200);
+  }
+
+  std::shared_ptr<Array> ArrayOf(std::shared_ptr<DataType> type, int64_t size,
+                                 double null_probability) {
+    return rag_.ArrayOf(std::move(type), size, null_probability);
+  }
+
+  // TODO(GH-38656): Use the random array generators from testing/random.h here

Review Comment:
   @pitrou I isolated all the random-generation code in this class and removed the complicated `List[View]ConcatenationChecker` templates.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/array/builder_nested.h:
##########
@@ -191,20 +191,129 @@ class BaseListBuilder : public ArrayBuilder {
     return std::make_shared<TYPE>(value_field_->WithType(value_builder_->type()));
   }
 
+ private:
+  static constexpr const char* type_name() {
+    if constexpr (is_list_view(TYPE::type_id)) {
+      return "ListView";
+    } else {
+      return "List";
+    }
+  }
+
  protected:
+  /// \brief Append dimensions for num_values empty list slots.
+  ///
+  /// ListViewBuilder overrides this to also append the sizes.
+  virtual void UnsafeAppendEmptyDimensions(int64_t num_values) {
+    const int64_t offset = value_builder_->length();
+    for (int64_t i = 0; i < num_values; ++i) {
+      offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+    }
+  }
+
+  /// \brief Append dimensions for a single list slot.
+  ///
+  /// ListViewBuilder overrides this to also append the size.
+  virtual void UnsafeAppendDimensions(int64_t offset, int64_t size) {
+    offsets_builder_.UnsafeAppend(static_cast<offset_type>(offset));
+  }
+
   TypedBufferBuilder<offset_type> offsets_builder_;
   std::shared_ptr<ArrayBuilder> value_builder_;
   std::shared_ptr<Field> value_field_;
+};
+
+// ----------------------------------------------------------------------
+// ListBuilder / LargeListBuilder
+
+template <typename TYPE>
+class ARROW_EXPORT BaseListBuilder : public VarLengthListLikeBuilder<TYPE> {
+ private:
+  using BASE = VarLengthListLikeBuilder<TYPE>;
+
+ public:
+  using TypeClass = TYPE;
+  using offset_type = typename BASE::offset_type;
+
+  using BASE::BASE;
+
+  using BASE::Append;
+
+  ~BaseListBuilder() override = default;
+
+  /// \brief Start a new variable-length list slot
+  ///
+  /// This function should be called before beginning to append elements to the
+  /// value builder
+  ///
+  /// Prefer Append(is_valid, 0) as that works correctly for list-view types
+  /// as well as list types.
+  Status Append(bool is_valid = true) { return BASE::Append(is_valid, 0); }
+
+  /// \brief Vector append
+  ///
+  /// If passed, valid_bytes is of equal length to values, and any zero byte
+  /// will be considered as a null for that slot
+  Status AppendValues(const offset_type* offsets, int64_t length,
+                      const uint8_t* valid_bytes = NULLPTR) {
+    ARROW_RETURN_NOT_OK(this->Reserve(length));
+    this->UnsafeAppendToBitmap(valid_bytes, length);
+    this->offsets_builder_.UnsafeAppend(offsets, length);
+    return Status::OK();
+  }
+
+  Status AppendValues(const offset_type* offsets, const offset_type* sizes,
+                      int64_t length, const uint8_t* valid_bytes) final {
+    // offsets are assumed to be valid, but the first lenght-1 sizes have to be
+    // consistent with the offsets to rule out the possibility that the caller
+    // is passing sizes that could work if building a list-view, but don't work
+    // on building a list that requires offsets to be non-decreasing.
+    if (sizes) {

Review Comment:
   > ...then the third size (3) is incorrect yet silently accepted.
   
   Yes. The validations aren't bullet-proof, just best-effort.
   
   > Can we perhaps guard the validation with #ifndef NDEBUG?
   
   Sure. The point of the validation is to catch some mistakes. Debug assert is fine -- `AppendValues` implementations put a lot of trust in the caller as they are.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/util/list_util.cc:
##########
@@ -0,0 +1,353 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstdint>
+#include <vector>
+
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_nested.h"
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit_run_reader.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/list_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/string.h"
+
+namespace arrow::list_util {
+
+namespace internal {
+
+namespace {
+
+using arrow::internal::checked_cast;
+using arrow::internal::ReverseSetBitRunReader;
+using arrow::internal::SetBitRunReader;
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+std::optional<int64_t> MinViewOffset(const ArraySpan& input) {
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Make an access to the sizes buffer only when strictly necessary.
+#define MINIMIZE_MIN_VIEW_OFFSET(i)             \
+  auto offset = offsets[i];                     \
+  if (min_offset.has_value()) {                 \
+    if (offset < *min_offset && sizes[i] > 0) { \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  } else {                                      \
+    if (sizes[i] > 0) {                         \
+      if (offset == 0) {                        \
+        return 0;                               \
+      }                                         \
+      min_offset = offset;                      \
+    }                                           \
+  }
+
+  std::optional<offset_type> min_offset;
+  if (validity == nullptr) {
+    for (int64_t i = 0; i < input.length; i++) {
+      MINIMIZE_MIN_VIEW_OFFSET(i);
+    }
+  } else {
+    SetBitRunReader reader(validity, input.offset, input.length);
+    while (true) {
+      const auto run = reader.NextRun();
+      if (run.length == 0) {
+        break;
+      }
+      for (int64_t i = run.position; i < run.position + run.length; ++i) {
+        MINIMIZE_MIN_VIEW_OFFSET(i);
+      }
+    }
+  }
+  return min_offset;
+
+#undef MINIMIZE_MIN_VIEW_OFFSET
+}
+
+/// \pre input.length() > 0 && input.null_count() != input.length()
+/// \param input A LIST_VIEW or LARGE_LIST_VIEW array
+template <typename offset_type>
+int64_t MaxViewEnd(const ArraySpan& input) {
+  constexpr auto kInt64Max = std::numeric_limits<int64_t>::max();
+  const auto values_length = input.child_data[0].length;
+
+  const uint8_t* validity = input.buffers[0].data;
+  const auto* offsets = input.GetValues<offset_type>(1);
+  const auto* sizes = input.GetValues<offset_type>(2);
+
+  // Early-exit: 64-bit overflow detected. This is not possible on a valid list-view,
+  // but we return the maximum possible value to avoid undefined behavior.

Review Comment:
   Pushing a commit that removes this check. I think I'm finally understanding where we are strict about validations and where we can assume valid as pre-conditions.



-- 
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-35344: [C++][Format] Implementation of the LIST_VIEW and LARGE_LIST_VIEW array formats [arrow]

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


##########
cpp/src/arrow/c/bridge_test.cc:
##########
@@ -1930,6 +2004,33 @@ TEST_F(TestSchemaImport, NestedList) {
   CheckImport(list(fixed_size_list(int8(), 3)));
 }
 
+TEST_F(TestSchemaImport, ListView) {
+  FillPrimitive(AddChild(), "c");
+  FillListLike("+vl");
+  CheckImport(list_view(int8()));
+
+  FillPrimitive(AddChild(), "s", "item", 0);
+  FillListLike("+vl");
+  CheckImport(list_view(field("item", int16(), /*nullable=*/false)));
+
+  // Large list-view
+  FillPrimitive(AddChild(), "s");
+  FillListLike("+vL");
+  CheckImport(large_list_view(int16()));
+}
+
+TEST_F(TestSchemaImport, NestedListView) {
+  FillPrimitive(AddChild(), "c");
+  FillListLike(AddChild(), "+vl");
+  FillListLike("+vL");
+  CheckImport(large_list_view(list_view(int8())));
+
+  FillPrimitive(AddChild(), "c");
+  FillListLike(AddChild(), "+w:3");
+  FillListLike("+vl");
+  CheckImport(list_view(fixed_size_list(int8(), 3)));
+}
+

Review Comment:
   Oh yes. I've now completed the test set based on all occurrences of `List` in this file. It looks like this:
   
   ```
   TEST_F(TestSchemaExport, ListView) {                TEST_F(TestSchemaExport, List) {
   TEST_F(TestArrayExport, ListView) {                 TEST_F(TestArrayExport, List) {
   TEST_F(TestArrayExport, ListViewSliced) {           TEST_F(TestArrayExport, ListSliced) {
   TEST_F(TestDeviceArrayExport, ListView) {           TEST_F(TestDeviceArrayExport, List) {
   TEST_F(TestDeviceArrayExport, ListViewSliced) {     TEST_F(TestDeviceArrayExport, ListSliced) {
   TEST_F(TestSchemaImport, ListView) {                TEST_F(TestSchemaImport, List) {
   TEST_F(TestSchemaImport, NestedListView) {          TEST_F(TestSchemaImport, NestedList) {
                                                       
   TEST_F(TestArrayImport, ListView) {                 TEST_F(TestArrayImport, List) {
   TEST_F(TestArrayImport, NestedListView) {           TEST_F(TestArrayImport, NestedList) {
   TEST_F(TestArrayImport, ListViewWithOffset) {       TEST_F(TestArrayImport, ListWithOffset) {
   TEST_F(TestArrayImport, ListViewNoError) {          TEST_F(TestArrayImport, ListError) {
   TEST_F(TestSchemaRoundtrip, ListView) {             TEST_F(TestSchemaRoundtrip, List) {
   TEST_F(TestArrayRoundtrip, ListView) {              TEST_F(TestArrayRoundtrip, List) {
   ```
   
   Pushing commit 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