You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "bkietz (via GitHub)" <gi...@apache.org> on 2024/03/18 18:24:56 UTC

[PR] add some range-for helpers [arrow-nanoarrow]

bkietz opened a new pull request, #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404

   All of this *should* be c++11
   
   ```diff
   -  auto arrow_array = ImportArray(&array, large_utf8());
   -  ARROW_EXPECT_OK(arrow_array);
   -
   -  auto builder = LargeStringBuilder();
   -  ARROW_EXPECT_OK(builder.Append("1234"));
   -  ARROW_EXPECT_OK(builder.AppendNulls(2));
   -  ARROW_EXPECT_OK(builder.Append("56789"));
   -  ARROW_EXPECT_OK(builder.AppendEmptyValue());
   -  auto expected_array = builder.Finish();
   -
   -  EXPECT_TRUE(arrow_array.ValueUnsafe()->Equals(expected_array.ValueUnsafe()));
   +  EXPECT_THAT(nanoarrow::ViewAsBytes<64>(&array),
   +              ElementsAre("1234"_v, kNull, kNull, "56789"_v, ""_v));
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1574933882


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +550,312 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};

Review Comment:
   I know it is a bit circular, but testing the "create by buffer, consume by View" direction for at least one instantation of each template in the user-facing header is what I had in mind. Creating by buffer is not too bad in C++ using the pattern from the benchmarks:
   
   https://github.com/apache/arrow-nanoarrow/blob/db6630b7770d33edc5227f60929a269a4aff8a04/dev/benchmarks/c/array_benchmark.cc#L36-L43
   
   ...plus the newly added helpers to create buffers from C++ sequences (which we can't use in the bechmarks or they won't build against the previous version):
   
   https://github.com/apache/arrow-nanoarrow/blob/db6630b7770d33edc5227f60929a269a4aff8a04/src/nanoarrow/nanoarrow_hpp_test.cc#L206-L234
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1532056876


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {

Review Comment:
   So maybe `ViewArrayAs` since it's intended for viewing arrays? If there's an STL term that would help a user understand what this is for that may also be useful to put in the name (e.g., is this an STL "sequence"? Maybe `ArrayAsSequenceOf<int_32>()`?)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1532060892


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};

Review Comment:
   Also, if there is some STL term that could be used in the name, I wonder if that would be useful. I'm not sure "view" is the verb I would use to describe what this is doing (although if that's the term C++ users expect, go 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


Re: [PR] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1539353363


##########
src/nanoarrow/utils_test.cc:
##########
@@ -538,3 +538,36 @@ TEST(DecimalTest, DecimalRoundtripBitshiftTest) {
 
   ArrowBufferReset(&buffer);
 }
+
+TEST(MaybeTest, ConstructionAndConversion) {
+  // NOTE these will not print nicely in GTest unless we write PrintTo,

Review Comment:
   Hmm...right now nanoarrow.hpp is mostly about digging a pit of success for not leaking memory and is pretty minimal. That said, the failure mode for tests that use it is currently not all that informative:
   
   ```
   [ctest] /Users/dewey/Desktop/rscratch/arrow-nanoarrow/src/nanoarrow/array_test.cc:577: Failure
   [ctest] Value of: nanoarrow::ViewArrayAs<uint32_t>(&array)
   [ctest] Expected: has 2 elements where
   [ctest] element #0 is equal to 8-byte object <02-00 00-00 01-00 00-00>,
   [ctest] element #1 is equal to 8-byte object <03-00 00-00 01-00 00-00>
   [ctest]   Actual: { 8-byte object <01-00 00-00 01-00 00-00>, 8-byte object <03-00 00-00 01-00 00-00> }, whose element #0 doesn't match
   ```
   
   I would perhaps lean towards `#include <iostream>` for the purposes of this PR, and before the release we can figure out how best to structure the C++ component. Maybe something like `src/nanoarrow/nanoarrow_cpp(hpp?)` could contain a few files with smaller scopes (e.g., the current `nanoarrow.hpp`), and `src/nanoarrow/nanoarrow.hpp` would include all of them? That would let anybody who cared keep a minimal scope but let most people benefit from the epic that this PR contains?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] feat: Add `ArrowArray` and `ArrowArrayStream` C++ iterators [arrow-nanoarrow]

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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1539391746


##########
src/nanoarrow/utils_test.cc:
##########
@@ -538,3 +538,36 @@ TEST(DecimalTest, DecimalRoundtripBitshiftTest) {
 
   ArrowBufferReset(&buffer);
 }
+
+TEST(MaybeTest, ConstructionAndConversion) {
+  // NOTE these will not print nicely in GTest unless we write PrintTo,

Review Comment:
   I think it would work to define PrintTo in a different header + compilation unit that you could include only in 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -222,6 +221,9 @@ class Unique {
   T* operator->() noexcept { return &data_; }
   const T* operator->() const noexcept { return &data_; }
 
+  /// \brief Check for validity
+  explicit operator bool() const { return data_.release != nullptr; }

Review Comment:
   Well, that can be a follow up then.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};

Review Comment:
   I was trying to let ViewStream use an existing ArrowError if one is already in scope, which would make an internal one redundant. I can assert the return code is checked in either case. I'll rewrite to what you suggest 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1532042591


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};
+
+class ViewStream {
+ public:
+  ViewStream(ArrowArrayStream* stream, ArrowErrorCode* code, ArrowError* error)
+      : range_{Next{this, stream, ArrowArray{}}}, code_{code}, error_{error} {}
+
+  ViewStream(ArrowArrayStream* stream, ErrorWithCode* error)
+      : ViewStream{stream, &error->code, error} {}
+
+ private:
+  struct Next {
+    ViewStream* self;
+    ArrowArrayStream* stream;
+    ArrowArray array;
+
+    ArrowArray* operator()() {
+      if (array.release) {
+        ArrowArrayRelease(&array);
+      }
+      *self->code_ = ArrowArrayStreamGetNext(stream, &array, self->error_);
+      if (*self->code_ != NANOARROW_OK) {
+        NANOARROW_DCHECK(array.release == nullptr);
+        return nullptr;
+      }
+      if (array.release == nullptr) {
+        return nullptr;
+      }
+      return &array;
+    }
+  };
+
+  InputRange<Next> range_;
+  ArrowError* error_;
+  ArrowErrorCode* code_;
+
+ public:
+  using iterator = typename InputRange<Next>::iterator;
+  iterator begin() { return range_.begin(); }
+  iterator end() { return range_.end(); }
+};
+
+#if NANOARROW_HAS_ZIP

Review Comment:
   Definitely! (Not great to have two versions floating around)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/utils_test.cc:
##########
@@ -538,3 +538,36 @@ TEST(DecimalTest, DecimalRoundtripBitshiftTest) {
 
   ArrowBufferReset(&buffer);
 }
+
+TEST(MaybeTest, ConstructionAndConversion) {
+  // NOTE these will not print nicely in GTest unless we write PrintTo,

Review Comment:
   ... or if it's not a problem, let nanoarrow.hpp `#include <iostream>`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +550,312 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};

Review Comment:
   This doesn't seem to handle templates too well. I've added tests for RandomAccessRange and InputRange as well. If we bring in ArrayFromJSON or something, we could also have meaningful tests for View*



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

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

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


Re: [PR] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1576596132


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -222,6 +221,9 @@ class Unique {
   T* operator->() noexcept { return &data_; }
   const T* operator->() const noexcept { return &data_; }
 
+  /// \brief Check for validity
+  explicit operator bool() const { return data_.release != nullptr; }

Review Comment:
   Hmm...if `buffer->data` is null it is still perfectly valid to call `ArrowBufferAppend()`, although it is also not holding on to any resources at that point (other than the memory for the `ArrowBuffer` itself). I'm not sure where that lines up in terms of C++ idioms (which I am very bad at spotting).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/utils_test.cc:
##########
@@ -538,3 +538,36 @@ TEST(DecimalTest, DecimalRoundtripBitshiftTest) {
 
   ArrowBufferReset(&buffer);
 }
+
+TEST(MaybeTest, ConstructionAndConversion) {
+  // NOTE these will not print nicely in GTest unless we write PrintTo,

Review Comment:
   It would, but I will still need to add a source file to house 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1532090260


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};

Review Comment:
   Got 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1539307665


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +550,312 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};

Review Comment:
   Sorry I missed this! I think it makes sense for anything user facing to have a test. There is `docker compose run --rm coverage` to check.



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

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

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


Re: [PR] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1573356589


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +550,312 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};

Review Comment:
   Thanks! I'm comfortable not adding tests, although if there are no tests it should probably live in `nanoarrow_testing.hpp` until we have the collective bandwidth to add 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1531439477


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};
+
+class ViewStream {
+ public:
+  ViewStream(ArrowArrayStream* stream, ArrowErrorCode* code, ArrowError* error)
+      : range_{Next{this, stream, ArrowArray{}}}, code_{code}, error_{error} {}
+
+  ViewStream(ArrowArrayStream* stream, ErrorWithCode* error)
+      : ViewStream{stream, &error->code, error} {}
+
+ private:
+  struct Next {
+    ViewStream* self;
+    ArrowArrayStream* stream;
+    ArrowArray array;

Review Comment:
   Could this be `UniqueArray`? That would collapse your `if (array.release) ...` to `array.reset()`. If it can't be, does `Next` need a deleter to ensure that `array` doesn't leak?



##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {

Review Comment:
   At first glance I wonder if `ViewAs` is too generic but I also don't have a better idea and I also love how it simplifies the tests (`ViewArrayAs`? But maybe this could also be used to view a `ArrowBufferView` or `ArrowBuffer`?)



##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -15,11 +15,21 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <stdexcept>
+#include <sys/wait.h>
 #include <string>
 #include <vector>
 
 #include "nanoarrow.h"
+#include "nanoarrow/nanoarrow_types.h"

Review Comment:
   I think you can remove this one (clangd often gets confused and adds it in). sys/wait.h also seems like it may have been added by accident?



##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};
+
+class ViewStream {
+ public:
+  ViewStream(ArrowArrayStream* stream, ArrowErrorCode* code, ArrowError* error)
+      : range_{Next{this, stream, ArrowArray{}}}, code_{code}, error_{error} {}
+
+  ViewStream(ArrowArrayStream* stream, ErrorWithCode* error)
+      : ViewStream{stream, &error->code, error} {}
+
+ private:
+  struct Next {
+    ViewStream* self;
+    ArrowArrayStream* stream;
+    ArrowArray array;
+
+    ArrowArray* operator()() {
+      if (array.release) {
+        ArrowArrayRelease(&array);
+      }
+      *self->code_ = ArrowArrayStreamGetNext(stream, &array, self->error_);
+      if (*self->code_ != NANOARROW_OK) {
+        NANOARROW_DCHECK(array.release == nullptr);
+        return nullptr;
+      }
+      if (array.release == nullptr) {
+        return nullptr;
+      }
+      return &array;
+    }
+  };
+
+  InputRange<Next> range_;
+  ArrowError* error_;
+  ArrowErrorCode* code_;
+
+ public:
+  using iterator = typename InputRange<Next>::iterator;
+  iterator begin() { return range_.begin(); }
+  iterator end() { return range_.end(); }
+};
+
+#if NANOARROW_HAS_ZIP

Review Comment:
   I love the Zip! I wonder if `dev/benchmarks/c/utils.h` would be a better home for this at the moment since that's where it can be used. The range-based iterator and the `ViewAs` have direct connections to nanoarrow C structs and so this header feels like a good place for them...these are generic helpers and I'm not sure that *this* header is the best home (for now).



##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {

Review Comment:
   I'm not sure I understand the significance of the 0 here?



##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};

Review Comment:
   The error handling is definitely the most annoying part of having the range-based for be perfect (in the absence of exceptions). It's not perfect, but
   
   ```cpp
   ViewStream viewer(stream, &error);
   for (ArrowArray* item : viewer) {
     // stuff
   }
   NANOARROW_RETURN_NOT_OK(viewer.code());
   ```
   
   ...is the same number of lines as 
   
   ```cpp
   ErrorWithCode error;
   for (ArrowArray* item : ViewStream(stream, &error)) {
     // stuff
   }
   NANOARROW_RETURN_NOT_OK(error.code);
   ```
   
   ...and invents one less concept. It would be a mega bonus if we could crash (in debug mode) if somebody forgets to check the return code (not checking the return code is a frequent source of crashes at unrelated lines of code that has been a bit of a struggle to communicate to users).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {

Review Comment:
   It's a hack to reuse ViewAsBytes to indicate fixed size binary. I'll just rename 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1532053166


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};

Review Comment:
   I think it can still use an error in scope since `ArrowError*` is commonly passed as an argument (viewer would just save the pointer, which can also be null if one doesn't exist).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +550,312 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};

Review Comment:
   I've added a test for Maybe, do you want tests for the range utils 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1534416373


##########
src/nanoarrow/utils_test.cc:
##########
@@ -538,3 +538,36 @@ TEST(DecimalTest, DecimalRoundtripBitshiftTest) {
 
   ArrowBufferReset(&buffer);
 }
+
+TEST(MaybeTest, ConstructionAndConversion) {
+  // NOTE these will not print nicely in GTest unless we write PrintTo,

Review Comment:
   Does defining `<<` work?
   
   https://github.com/apache/arrow-nanoarrow/blob/dc50114756b7e9067b42181a6a86f928effc6e68/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_files_test.cc#L309-L312



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {

Review Comment:
   If it views a single buffer it can't detect null bits



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -222,6 +221,9 @@ class Unique {
   T* operator->() noexcept { return &data_; }
   const T* operator->() const noexcept { return &data_; }
 
+  /// \brief Check for validity
+  explicit operator bool() const { return data_.release != nullptr; }

Review Comment:
   Fair enough; I'll remove this. Unless there are validity conditions for all the other objects wrapped by Unique which I could specialize this for? For example, in the case of `nanoarrow::UniqueBuffer` I could check whether `buffer->data` is null



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#issuecomment-2004648609

   ## [Codecov](https://app.codecov.io/gh/apache/arrow-nanoarrow/pull/404?dropdown=coverage&src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   All modified and coverable lines are covered by tests :white_check_mark:
   > Project coverage is 88.79%. Comparing base [(`dc50114`)](https://app.codecov.io/gh/apache/arrow-nanoarrow/commit/dc50114756b7e9067b42181a6a86f928effc6e68?dropdown=coverage&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) to head [(`df6bc4c`)](https://app.codecov.io/gh/apache/arrow-nanoarrow/pull/404?dropdown=coverage&src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@            Coverage Diff             @@
   ##             main     #404      +/-   ##
   ==========================================
   + Coverage   88.74%   88.79%   +0.04%     
   ==========================================
     Files          81       81              
     Lines       14398    14457      +59     
   ==========================================
   + Hits        12778    12837      +59     
     Misses       1620     1620              
   ```
   
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/arrow-nanoarrow/pull/404?dropdown=coverage&src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1533142084


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +550,312 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};

Review Comment:
   Do you have bandwidth to add tests for these? (I can try also)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1532088340


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};

Review Comment:
   Presumably `view` comes from ranges: https://en.cppreference.com/w/cpp/ranges/view



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

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

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


Re: [PR] add some range-for helpers [arrow-nanoarrow]

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


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -548,6 +558,366 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_convertible<ValueOrFalsy, bool>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      if (stashed) false;
+      // The stashed value is falsy, so the LHS iterator is at the end.
+      // Check if RHS iterator is the end sentinel.
+      return other.range == nullptr;
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this}; }
+  iterator end() { return {}; }
+};
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize = 0>
+class ViewAsBytes {
+ private:
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <>
+class ViewAsBytes<0> {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename RandomAccessRange<Get>::value_type;
+  using const_iterator = typename RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+struct ErrorWithCode : ArrowError {
+  ErrorWithCode() : ArrowError{}, code{NANOARROW_OK} {}
+  ArrowErrorCode code;
+  constexpr operator bool() const { return code != NANOARROW_OK; }
+};
+
+class ViewStream {
+ public:
+  ViewStream(ArrowArrayStream* stream, ArrowErrorCode* code, ArrowError* error)
+      : range_{Next{this, stream, ArrowArray{}}}, code_{code}, error_{error} {}
+
+  ViewStream(ArrowArrayStream* stream, ErrorWithCode* error)
+      : ViewStream{stream, &error->code, error} {}
+
+ private:
+  struct Next {
+    ViewStream* self;
+    ArrowArrayStream* stream;
+    ArrowArray array;
+
+    ArrowArray* operator()() {
+      if (array.release) {
+        ArrowArrayRelease(&array);
+      }
+      *self->code_ = ArrowArrayStreamGetNext(stream, &array, self->error_);
+      if (*self->code_ != NANOARROW_OK) {
+        NANOARROW_DCHECK(array.release == nullptr);
+        return nullptr;
+      }
+      if (array.release == nullptr) {
+        return nullptr;
+      }
+      return &array;
+    }
+  };
+
+  InputRange<Next> range_;
+  ArrowError* error_;
+  ArrowErrorCode* code_;
+
+ public:
+  using iterator = typename InputRange<Next>::iterator;
+  iterator begin() { return range_.begin(); }
+  iterator end() { return range_.end(); }
+};
+
+#if NANOARROW_HAS_ZIP

Review Comment:
   After I move this, should I remove the preprocessor conditions in favor of raising an error if !NANOARROW_HAS_ZIP? https://github.com/apache/arrow-nanoarrow/pull/404/files#diff-52c5c898fc0e86beb4c6c267d8cc09469f26695a2e401b4ee6c49613226432f8R106



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] add some range-for helpers [arrow-nanoarrow]

Posted by "paleolimbot (via GitHub)" <gi...@apache.org>.
paleolimbot commented on code in PR #404:
URL: https://github.com/apache/arrow-nanoarrow/pull/404#discussion_r1573358476


##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -550,6 +552,314 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+namespace internal {
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_constructible<bool, ValueOrFalsy>::value, "");
+  static_assert(std::is_default_constructible<ValueOrFalsy>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      return static_cast<bool>(stashed) == static_cast<bool>(other.stashed);
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this, next()}; }
+  iterator end() { return {this, {}}; }
+};
+}  // namespace internal
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewArrayAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  internal::RandomAccessRange<Get> range_;
+
+ public:
+  ViewArrayAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewArrayAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename internal::RandomAccessRange<Get>::value_type;
+  using const_iterator = typename internal::RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize>
+class ViewArrayAsBytes {
+ private:
+  static_assert(OffsetSize == 32 || OffsetSize == 64, "");
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  internal::RandomAccessRange<Get> range_;
+
+ public:
+  ViewArrayAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewArrayAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename internal::RandomAccessRange<Get>::value_type;
+  using const_iterator = typename internal::RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+class ViewAsFixedSizeBytes {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  internal::RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsFixedSizeBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsFixedSizeBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename internal::RandomAccessRange<Get>::value_type;
+  using const_iterator = typename internal::RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+class ViewArrayStream {
+ public:
+  ViewArrayStream(ArrowArrayStream* stream, ArrowErrorCode* code, ArrowError* error)
+      : range_{Next{this, stream, UniqueArray{}}}, code_{code}, error_{error} {}
+
+  ViewArrayStream(ArrowArrayStream* stream, ArrowError* error)
+      : ViewArrayStream{stream, &internal_code_, error} {}
+
+  ViewArrayStream(ArrowArrayStream* stream)
+      : ViewArrayStream{stream, &internal_code_, &internal_error_} {}
+
+  // disable copy/move of this view, since its error references may point into itself
+  ViewArrayStream(ViewArrayStream&&) = delete;
+  ViewArrayStream& operator=(ViewArrayStream&&) = delete;
+  ViewArrayStream(const ViewArrayStream&) = delete;
+  ViewArrayStream& operator=(const ViewArrayStream&) = delete;
+
+  // ensure the error code of this stream was accessed at least once
+  ~ViewArrayStream() { NANOARROW_DCHECK(code_was_accessed_); }

Review Comment:
   Thank you for adding this bit!



##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -222,6 +221,9 @@ class Unique {
   T* operator->() noexcept { return &data_; }
   const T* operator->() const noexcept { return &data_; }
 
+  /// \brief Check for validity
+  explicit operator bool() const { return data_.release != nullptr; }

Review Comment:
   This seems like it might be confusing for something like `nanoarrow::UniqueBuffer` that doesn't have a `.release` callback?



##########
src/nanoarrow/array_test.cc:
##########
@@ -30,8 +32,11 @@
 #include <arrow/util/decimal.h>
 
 #include "nanoarrow/nanoarrow.h"
+#include "nanoarrow/nanoarrow.hpp"

Review Comment:
   ```suggestion
   #include "nanoarrow/nanoarrow.hpp"
   ```



##########
dist/nanoarrow_testing.hpp:
##########


Review Comment:
   There's a nightly job that takes care of updating this (although it's not a problem to leave this change in since the nightly job will overwrite it anyway).



##########
src/nanoarrow/array_stream_test.cc:
##########
@@ -15,9 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <gmock/gmock-matchers.h>
 #include <gtest/gtest.h>
 
 #include "nanoarrow/nanoarrow.h"
+#include "nanoarrow/nanoarrow.hpp"

Review Comment:
   ```suggestion
   #include "nanoarrow/nanoarrow.hpp"
   ```



##########
src/nanoarrow/nanoarrow.hpp:
##########
@@ -550,6 +552,314 @@ class VectorArrayStream {
 
 /// @}
 
+struct Nothing {};
+constexpr Nothing NA{};
+
+template <typename T>
+class Maybe {
+ public:
+  Maybe() : nothing_{Nothing{}}, is_something_{false} {}
+  Maybe(Nothing) : Maybe{} {}
+
+  Maybe(T something)  // NOLINT(google-explicit-constructor)
+      : something_{something}, is_something_{true} {}
+
+  explicit constexpr operator bool() const { return is_something_; }
+
+  const T& operator*() const { return something_; }
+
+  friend inline bool operator==(Maybe l, Maybe r) {
+    if (l.is_something_ != r.is_something_) return false;
+    return l.is_something_ ? l.something_ == r.something_ : true;
+  }
+  friend inline bool operator!=(Maybe l, Maybe r) { return !(l == r); }
+
+  T value_or(T val) const { return is_something_ ? something_ : val; }
+
+ private:
+  static_assert(std::is_trivially_copyable<T>::value, "");
+  static_assert(std::is_trivially_destructible<T>::value, "");
+
+  union {
+    Nothing nothing_;
+    T something_;
+  };
+  bool is_something_;
+};
+
+namespace internal {
+template <typename Get>
+struct RandomAccessRange {
+  Get get;
+  int64_t size;
+
+  using value_type = decltype(std::declval<Get>()(0));
+
+  struct const_iterator {
+    int64_t i;
+    const RandomAccessRange* range;
+    bool operator==(const_iterator other) const { return i == other.i; }
+    bool operator!=(const_iterator other) const { return i != other.i; }
+    const_iterator& operator++() { return ++i, *this; }
+    value_type operator*() const { return range->get(i); }
+  };
+
+  const_iterator begin() const { return {0, this}; }
+  const_iterator end() const { return {size, this}; }
+};
+
+template <typename Next>
+struct InputRange {
+  Next next;
+  using ValueOrFalsy = decltype(std::declval<Next>()());
+
+  static_assert(std::is_constructible<bool, ValueOrFalsy>::value, "");
+  static_assert(std::is_default_constructible<ValueOrFalsy>::value, "");
+  using value_type = decltype(*std::declval<ValueOrFalsy>());
+
+  struct iterator {
+    InputRange* range;
+    ValueOrFalsy stashed;
+
+    bool operator==(iterator other) const {
+      return static_cast<bool>(stashed) == static_cast<bool>(other.stashed);
+    }
+    bool operator!=(iterator other) const { return !(*this == other); }
+
+    iterator& operator++() {
+      stashed = range->next();
+      return *this;
+    }
+    value_type operator*() const { return *stashed; }
+  };
+
+  iterator begin() { return {this, next()}; }
+  iterator end() { return {this, {}}; }
+};
+}  // namespace internal
+
+// C++17 could do all of this with a lambda
+template <typename T>
+class ViewArrayAs {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const void* values;
+    int64_t offset;
+
+    Maybe<T> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        if (std::is_same<T, bool>::value) {
+          return ArrowBitGet(static_cast<const uint8_t*>(values), i);
+        } else {
+          return static_cast<const T*>(values)[i];
+        }
+      }
+      return Nothing{};
+    }
+  };
+
+  internal::RandomAccessRange<Get> range_;
+
+ public:
+  ViewArrayAs(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewArrayAs(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename internal::RandomAccessRange<Get>::value_type;
+  using const_iterator = typename internal::RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+template <int OffsetSize>
+class ViewArrayAsBytes {
+ private:
+  static_assert(OffsetSize == 32 || OffsetSize == 64, "");
+  using OffsetType = typename std::conditional<OffsetSize == 32, int32_t, int64_t>::type;
+
+  struct Get {
+    const uint8_t* validity;
+    const void* offsets;
+    const char* data;
+    int64_t offset;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      auto* offsets = static_cast<const OffsetType*>(this->offsets);
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + offsets[i], offsets[i + 1] - offsets[i]};
+      }
+      return Nothing{};
+    }
+  };
+
+  internal::RandomAccessRange<Get> range_;
+
+ public:
+  ViewArrayAsBytes(const ArrowArrayView* array_view)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.data,
+                array_view->buffer_views[2].data.as_char,
+                array_view->offset,
+            },
+            array_view->length,
+        } {}
+
+  ViewArrayAsBytes(const ArrowArray* array)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                array->buffers[1],
+                static_cast<const char*>(array->buffers[2]),
+                /*offset=*/0,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename internal::RandomAccessRange<Get>::value_type;
+  using const_iterator = typename internal::RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+class ViewAsFixedSizeBytes {
+ private:
+  struct Get {
+    const uint8_t* validity;
+    const char* data;
+    int64_t offset;
+    int fixed_size;
+
+    Maybe<ArrowStringView> operator()(int64_t i) const {
+      i += offset;
+      if (validity == nullptr || ArrowBitGet(validity, i)) {
+        return ArrowStringView{data + i * fixed_size, fixed_size};
+      }
+      return Nothing{};
+    }
+  };
+
+  internal::RandomAccessRange<Get> range_;
+
+ public:
+  ViewAsFixedSizeBytes(const ArrowArrayView* array_view, int fixed_size)
+      : range_{
+            Get{
+                array_view->buffer_views[0].data.as_uint8,
+                array_view->buffer_views[1].data.as_char,
+                array_view->offset,
+                fixed_size,
+            },
+            array_view->length,
+        } {}
+
+  ViewAsFixedSizeBytes(const ArrowArray* array, int fixed_size)
+      : range_{
+            Get{
+                static_cast<const uint8_t*>(array->buffers[0]),
+                static_cast<const char*>(array->buffers[1]),
+                /*offset=*/0,
+                fixed_size,
+            },
+            array->length,
+        } {}
+
+  using value_type = typename internal::RandomAccessRange<Get>::value_type;
+  using const_iterator = typename internal::RandomAccessRange<Get>::const_iterator;
+  const_iterator begin() const { return range_.begin(); }
+  const_iterator end() const { return range_.end(); }
+  value_type operator[](int64_t i) const { return range_.get(i); }
+};
+
+class ViewArrayStream {
+ public:
+  ViewArrayStream(ArrowArrayStream* stream, ArrowErrorCode* code, ArrowError* error)
+      : range_{Next{this, stream, UniqueArray{}}}, code_{code}, error_{error} {}
+
+  ViewArrayStream(ArrowArrayStream* stream, ArrowError* error)
+      : ViewArrayStream{stream, &internal_code_, error} {}
+
+  ViewArrayStream(ArrowArrayStream* stream)
+      : ViewArrayStream{stream, &internal_code_, &internal_error_} {}
+
+  // disable copy/move of this view, since its error references may point into itself
+  ViewArrayStream(ViewArrayStream&&) = delete;
+  ViewArrayStream& operator=(ViewArrayStream&&) = delete;
+  ViewArrayStream(const ViewArrayStream&) = delete;
+  ViewArrayStream& operator=(const ViewArrayStream&) = delete;
+
+  // ensure the error code of this stream was accessed at least once
+  ~ViewArrayStream() { NANOARROW_DCHECK(code_was_accessed_); }
+
+ private:
+  struct Next {
+    ViewArrayStream* self;
+    ArrowArrayStream* stream;
+    UniqueArray array;
+
+    ArrowArray* operator()() {
+      array.reset();
+      *self->code_ = ArrowArrayStreamGetNext(stream, array.get(), self->error_);
+      NANOARROW_DCHECK(
+          // either there was no error or there was an error and array is invalid
+          *self->code_ == NANOARROW_OK || !array);
+      return array ? array.get() : nullptr;
+    }
+  };
+
+  internal::InputRange<Next> range_;
+  ArrowError* error_;
+  ArrowErrorCode* code_;
+  ArrowError internal_error_ = {};
+  ArrowErrorCode internal_code_;
+  bool code_was_accessed_;
+  // ArrowErrorCode internal_code_ = NANOARROW_OK;
+  // bool code_was_accessed_ = false;

Review Comment:
   Do these need to be removed?



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

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

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