You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "pitrou (via GitHub)" <gi...@apache.org> on 2023/09/21 15:59:07 UTC

[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

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