You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/11/20 22:23:41 UTC

[GitHub] [arrow] bkietz commented on a change in pull request #8703: ARROW-10143: [C++] Rewrite Array(Range)Equals

bkietz commented on a change in pull request #8703:
URL: https://github.com/apache/arrow/pull/8703#discussion_r526259632



##########
File path: cpp/src/arrow/ipc/feather_test.cc
##########
@@ -286,10 +286,13 @@ TEST_P(TestFeather, PrimitiveNullRoundTrip) {
     std::vector<std::shared_ptr<Array>> expected_fields;
     for (int i = 0; i < batch->num_columns(); ++i) {
       ASSERT_EQ(batch->column_name(i), reader_->schema()->field(i)->name());
-      StringArray str_values(batch->column(i)->length(), nullptr, nullptr,
-                             batch->column(i)->null_bitmap(),
-                             batch->column(i)->null_count());
-      AssertArraysEqual(str_values, *result->column(i)->chunk(0));
+      ASSERT_OK_AND_ASSIGN(auto expected, MakeArrayOfNull(utf8(), batch->num_rows()));
+      AssertArraysEqual(*expected, *result->column(i)->chunk(0));
+      //       StringArray str_values(batch->column(i)->length(), nullptr, nullptr,
+      //                              batch->column(i)->null_bitmap(),
+      //                              batch->column(i)->null_count());
+      //       AssertArraysEqual(str_values, *result->column(i)->chunk(0),
+      //       /*verbose=*/true);

Review comment:
       Looks like leftovers
   ```suggestion
   ```

##########
File path: cpp/src/arrow/util/bit_run_reader.h
##########
@@ -162,5 +166,7 @@ class ARROW_EXPORT BitRunReader {
 using BitRunReader = BitRunReaderLinear;
 #endif
 
+// TODO SetBitRunReader?
+

Review comment:
       IIUC BitRunReader yields alternating set/unset BitRuns. Would SetBitRunReader yield only the set BitRuns?

##########
File path: cpp/src/arrow/compare.cc
##########
@@ -49,700 +51,441 @@
 namespace arrow {
 
 using internal::BitmapEquals;
+using internal::BitmapReader;
+using internal::BitmapUInt64Reader;
 using internal::checked_cast;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitmapEquals;
 
 // ----------------------------------------------------------------------
 // Public method implementations
 
 namespace {
 
-// These helper functions assume we already checked the arrays have equal
-// sizes and null bitmaps.
+bool CompareArrayRanges(const ArrayData& left, const ArrayData& right,
+                        int64_t left_start_idx, int64_t left_end_idx,
+                        int64_t right_start_idx, const EqualOptions& options,
+                        bool floating_approximate);
 
-template <typename ArrowType, typename EqualityFunc>
-inline bool BaseFloatingEquals(const NumericArray<ArrowType>& left,
-                               const NumericArray<ArrowType>& right,
-                               EqualityFunc&& equals) {
-  using T = typename ArrowType::c_type;
-
-  const T* left_data = left.raw_values();
-  const T* right_data = right.raw_values();
-
-  if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i)) continue;
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  } else {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  }
-  return true;
-}
-
-template <typename ArrowType>
-inline bool FloatingEquals(const NumericArray<ArrowType>& left,
-                           const NumericArray<ArrowType>& right,
-                           const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [](T x, T y) -> bool {
-      return (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right,
-                                         [](T x, T y) -> bool { return x == y; });
-  }
-}
-
-template <typename ArrowType>
-inline bool FloatingApproxEquals(const NumericArray<ArrowType>& left,
-                                 const NumericArray<ArrowType>& right,
-                                 const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-  const T epsilon = static_cast<T>(opts.atol());
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y);
-    });
-  }
-}
-
-// RangeEqualsVisitor assumes the range sizes are equal
-
-class RangeEqualsVisitor {
+class RangeDataEqualsImpl {
  public:
-  RangeEqualsVisitor(const Array& right, int64_t left_start_idx, int64_t left_end_idx,
-                     int64_t right_start_idx)
-      : right_(right),
+  // PRE-CONDITIONS:
+  // - the types are equal
+  // - the ranges are in bounds
+  RangeDataEqualsImpl(const EqualOptions& options, bool floating_approximate,
+                      const ArrayData& left, const ArrayData& right,
+                      int64_t left_start_idx, int64_t right_start_idx,
+                      int64_t range_length)
+      : options_(options),
+        floating_approximate_(floating_approximate),
+        left_(left),
+        right_(right),
         left_start_idx_(left_start_idx),
-        left_end_idx_(left_end_idx),
         right_start_idx_(right_start_idx),
+        range_length_(range_length),
         result_(false) {}
 
-  template <typename ArrayType>
-  inline Status CompareValues(const ArrayType& left) {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i) ||
-          (!is_null && left.Value(i) != right.Value(o_i))) {
-        result_ = false;
-        return Status::OK();
+  bool Compare() {
+    // Compare null bitmaps
+    if (left_start_idx_ == 0 && right_start_idx_ == 0 && range_length_ == left_.length &&
+        range_length_ == right_.length) {
+      // If we're comparing entire arrays, we can first compare the cached null counts
+      if (left_.GetNullCount() != right_.GetNullCount()) {
+        return false;
       }
     }

Review comment:
       I'm not sure why this optimization only applies to entire arrays. Additionally, as currently stated it doesn't necessarily compare cached null counts.
   
   If we want to absolutely avoid accessing the null bitmap here, we'll need:
   ```suggestion
       // Compare null bitmaps
       // Try to compare cached null counts first:
       int64_t left_null_count = left_.null_count.load(), right_null_count = right_.null_count.load();
       if (left_null_count != kUnknownNullCount && right_null_count != kUnknownNullCount &&
           left_null_count != right_null_count) {
         return false;
       }
   ```

##########
File path: cpp/src/arrow/compare.cc
##########
@@ -49,700 +51,441 @@
 namespace arrow {
 
 using internal::BitmapEquals;
+using internal::BitmapReader;
+using internal::BitmapUInt64Reader;
 using internal::checked_cast;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitmapEquals;
 
 // ----------------------------------------------------------------------
 // Public method implementations
 
 namespace {
 
-// These helper functions assume we already checked the arrays have equal
-// sizes and null bitmaps.
+bool CompareArrayRanges(const ArrayData& left, const ArrayData& right,
+                        int64_t left_start_idx, int64_t left_end_idx,
+                        int64_t right_start_idx, const EqualOptions& options,
+                        bool floating_approximate);
 
-template <typename ArrowType, typename EqualityFunc>
-inline bool BaseFloatingEquals(const NumericArray<ArrowType>& left,
-                               const NumericArray<ArrowType>& right,
-                               EqualityFunc&& equals) {
-  using T = typename ArrowType::c_type;
-
-  const T* left_data = left.raw_values();
-  const T* right_data = right.raw_values();
-
-  if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i)) continue;
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  } else {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  }
-  return true;
-}
-
-template <typename ArrowType>
-inline bool FloatingEquals(const NumericArray<ArrowType>& left,
-                           const NumericArray<ArrowType>& right,
-                           const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [](T x, T y) -> bool {
-      return (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right,
-                                         [](T x, T y) -> bool { return x == y; });
-  }
-}
-
-template <typename ArrowType>
-inline bool FloatingApproxEquals(const NumericArray<ArrowType>& left,
-                                 const NumericArray<ArrowType>& right,
-                                 const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-  const T epsilon = static_cast<T>(opts.atol());
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y);
-    });
-  }
-}
-
-// RangeEqualsVisitor assumes the range sizes are equal
-
-class RangeEqualsVisitor {
+class RangeDataEqualsImpl {
  public:
-  RangeEqualsVisitor(const Array& right, int64_t left_start_idx, int64_t left_end_idx,
-                     int64_t right_start_idx)
-      : right_(right),
+  // PRE-CONDITIONS:
+  // - the types are equal
+  // - the ranges are in bounds
+  RangeDataEqualsImpl(const EqualOptions& options, bool floating_approximate,
+                      const ArrayData& left, const ArrayData& right,
+                      int64_t left_start_idx, int64_t right_start_idx,
+                      int64_t range_length)
+      : options_(options),
+        floating_approximate_(floating_approximate),
+        left_(left),
+        right_(right),
         left_start_idx_(left_start_idx),
-        left_end_idx_(left_end_idx),
         right_start_idx_(right_start_idx),
+        range_length_(range_length),
         result_(false) {}
 
-  template <typename ArrayType>
-  inline Status CompareValues(const ArrayType& left) {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i) ||
-          (!is_null && left.Value(i) != right.Value(o_i))) {
-        result_ = false;
-        return Status::OK();
+  bool Compare() {
+    // Compare null bitmaps
+    if (left_start_idx_ == 0 && right_start_idx_ == 0 && range_length_ == left_.length &&
+        range_length_ == right_.length) {
+      // If we're comparing entire arrays, we can first compare the cached null counts
+      if (left_.GetNullCount() != right_.GetNullCount()) {
+        return false;
       }
     }
-    result_ = true;
-    return Status::OK();
+    if (!OptionalBitmapEquals(left_.buffers[0], left_.offset + left_start_idx_,
+                              right_.buffers[0], right_.offset + right_start_idx_,
+                              range_length_)) {
+      return false;
+    }
+    // Compare values
+    return CompareWithType(*left_.type);
   }
 
-  template <typename ArrayType, typename CompareValuesFunc>
-  bool CompareWithOffsets(const ArrayType& left,
-                          CompareValuesFunc&& compare_values) const {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        return false;
-      }
-      if (is_null) continue;
-      const auto begin_offset = left.value_offset(i);
-      const auto end_offset = left.value_offset(i + 1);
-      const auto right_begin_offset = right.value_offset(o_i);
-      const auto right_end_offset = right.value_offset(o_i + 1);
-      // Underlying can't be equal if the size isn't equal
-      if (end_offset - begin_offset != right_end_offset - right_begin_offset) {
-        return false;
-      }
-
-      if (!compare_values(left, right, begin_offset, right_begin_offset,
-                          end_offset - begin_offset)) {
-        return false;
-      }
+  bool CompareWithType(const DataType& type) {
+    result_ = true;
+    if (range_length_ != 0) {
+      ARROW_CHECK_OK(VisitTypeInline(type, this));
     }
-    return true;
+    return result_;
   }
 
-  template <typename BinaryArrayType>
-  bool CompareBinaryRange(const BinaryArrayType& left) const {
-    using offset_type = typename BinaryArrayType::offset_type;
+  Status Visit(const NullType&) { return Status::OK(); }
 
-    auto compare_values = [](const BinaryArrayType& left, const BinaryArrayType& right,
-                             offset_type left_offset, offset_type right_offset,
-                             offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return std::memcmp(left.value_data()->data() + left_offset,
-                         right.value_data()->data() + right_offset,
-                         static_cast<size_t>(nvalues)) == 0;
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_primitive_ctype<TypeClass, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  template <typename ListArrayType>
-  bool CompareLists(const ListArrayType& left) {
-    using offset_type = typename ListArrayType::offset_type;
-    const auto& right = checked_cast<const ListArrayType&>(right_);
-    const std::shared_ptr<Array>& left_values = left.values();
-    const std::shared_ptr<Array>& right_values = right.values();
-
-    auto compare_values = [&](const ListArrayType& left, const ListArrayType& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_values->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                      right_values);
-    };
-    return CompareWithOffsets(left, compare_values);
-  }
-
-  bool CompareMaps(const MapArray& left) {
-    // We need a specific comparison helper for maps to avoid comparing
-    // struct field names (which are indifferent for maps)
-    using offset_type = typename MapArray::offset_type;
-    const auto& right = checked_cast<const MapArray&>(right_);
-    const auto left_keys = left.keys();
-    const auto left_items = left.items();
-    const auto right_keys = right.keys();
-    const auto right_items = right.items();
-
-    auto compare_values = [&](const MapArray& left, const MapArray& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_keys->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                    right_keys) &&
-             left_items->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                     right_items);
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_t<is_temporal_type<TypeClass>::value, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  bool CompareStructs(const StructArray& left) {
-    const auto& right = checked_cast<const StructArray&>(right_);
-    bool equal_fields = true;
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      for (int j = 0; j < left.num_fields(); ++j) {
-        // TODO: really we should be comparing stretches of non-null data rather
-        // than looking at one value at a time.
-        equal_fields = left.field(j)->RangeEquals(i, i + 1, o_i, right.field(j));
-        if (!equal_fields) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  bool CompareUnions(const UnionArray& left) const {
-    const auto& right = checked_cast<const UnionArray&>(right_);
-
-    const UnionMode::type union_mode = left.mode();
-    if (union_mode != right.mode()) {
-      return false;
-    }
-
-    const auto& left_type = checked_cast<const UnionType&>(*left.type());
-
-    const std::vector<int>& child_ids = left_type.child_ids();
-
-    const int8_t* left_codes = left.raw_type_codes();
-    const int8_t* right_codes = right.raw_type_codes();
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      if (left_codes[i] != right_codes[o_i]) {
-        return false;
-      }
-
-      auto child_num = child_ids[left_codes[i]];
-
-      // TODO(wesm): really we should be comparing stretches of non-null data
-      // rather than looking at one value at a time.
-      if (union_mode == UnionMode::SPARSE) {
-        if (!left.field(child_num)->RangeEquals(i, i + 1, o_i, right.field(child_num))) {
-          return false;
+  Status Visit(const BooleanType&) {
+    const uint8_t* left_bits = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_bits = right_.GetValues<uint8_t>(1, 0);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      if (length <= 8) {
+        // Avoid the BitmapUInt64Reader overhead for very small runs
+        for (int64_t j = i; j < i + length; ++j) {
+          if (BitUtil::GetBit(left_bits, left_start_idx_ + left_.offset + j) !=
+              BitUtil::GetBit(right_bits, right_start_idx_ + right_.offset + j)) {
+            return false;
+          }
         }
+        return true;
       } else {
-        const int32_t offset =
-            checked_cast<const DenseUnionArray&>(left).raw_value_offsets()[i];
-        const int32_t o_offset =
-            checked_cast<const DenseUnionArray&>(right).raw_value_offsets()[o_i];
-        if (!left.field(child_num)->RangeEquals(offset, offset + 1, o_offset,
-                                                right.field(child_num))) {
-          return false;
+        BitmapUInt64Reader left_reader(left_bits, left_start_idx_ + left_.offset + i,
+                                       length);
+        BitmapUInt64Reader right_reader(right_bits, right_start_idx_ + right_.offset + i,
+                                        length);
+        while (left_reader.position() < length) {
+          if (left_reader.NextWord() != right_reader.NextWord()) {
+            return false;
+          }
         }
+        DCHECK_EQ(right_reader.position(), length);
       }
-    }
-    return true;
-  }
-
-  Status Visit(const BinaryArray& left) {
-    result_ = CompareBinaryRange(left);
-    return Status::OK();
-  }
-
-  Status Visit(const LargeBinaryArray& left) {
-    result_ = CompareBinaryRange(left);
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const FixedSizeBinaryArray& left) {
-    const auto& right = checked_cast<const FixedSizeBinaryArray&>(right_);
+  Status Visit(const FloatType& type) { return CompareFloating(type); }
 
-    int32_t width = left.byte_width();
+  Status Visit(const DoubleType& type) { return CompareFloating(type); }
 
-    const uint8_t* left_data = nullptr;
-    const uint8_t* right_data = nullptr;
+  // Also matches StringType
+  Status Visit(const BinaryType& type) { return CompareBinary(type); }
 
-    if (left.values()) {
-      left_data = left.raw_values();
-    }
+  // Also matches LargeStringType
+  Status Visit(const LargeBinaryType& type) { return CompareBinary(type); }
 
-    if (right.values()) {
-      right_data = right.raw_values();
-    }
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        result_ = false;
-        return Status::OK();
-      }
-      if (is_null) continue;
+  Status Visit(const FixedSizeBinaryType& type) {
+    const auto byte_width = type.byte_width();
+    const uint8_t* left_data = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_data = right_.GetValues<uint8_t>(1, 0);
 
-      if (std::memcmp(left_data + width * i, right_data + width * o_i, width)) {
-        result_ = false;
-        return Status::OK();
-      }
+    if (left_data != nullptr && right_data != nullptr) {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+        return memcmp(left_data + (left_start_idx_ + left_.offset + i) * byte_width,
+                      right_data + (right_start_idx_ + right_.offset + i) * byte_width,
+                      length * byte_width) == 0;
+      };
+      VisitValidRuns(compare_runs);
+    } else {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool { return true; };
+      VisitValidRuns(compare_runs);
     }
-    result_ = true;
     return Status::OK();
   }
 
-  Status Visit(const Decimal128Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  // Also matches MapType
+  Status Visit(const ListType& type) { return CompareList(type); }
 
-  Status Visit(const Decimal256Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  Status Visit(const LargeListType& type) { return CompareList(type); }
 
-  Status Visit(const NullArray& left) {
-    ARROW_UNUSED(left);
-    result_ = true;
-    return Status::OK();
-  }
-
-  template <typename T>
-  typename std::enable_if<std::is_base_of<PrimitiveArray, T>::value, Status>::type Visit(
-      const T& left) {
-    return CompareValues<T>(left);
-  }
+  Status Visit(const FixedSizeListType& type) {
+    const auto list_size = type.list_size();
+    const ArrayData& left_data = *left_.child_data[0];
+    const ArrayData& right_data = *right_.child_data[0];
 
-  Status Visit(const ListArray& left) {
-    result_ = CompareLists(left);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      RangeDataEqualsImpl impl(options_, floating_approximate_, left_data, right_data,
+                               (left_start_idx_ + left_.offset + i) * list_size,
+                               (right_start_idx_ + right_.offset + i) * list_size,
+                               length * list_size);
+      return impl.Compare();
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const LargeListArray& left) {
-    result_ = CompareLists(left);
-    return Status::OK();
-  }
+  Status Visit(const StructType& type) {
+    const int32_t num_fields = type.num_fields();
 
-  Status Visit(const FixedSizeListArray& left) {
-    const auto& right = checked_cast<const FixedSizeListArray&>(right_);
-    result_ = left.values()->RangeEquals(
-        left.value_offset(left_start_idx_), left.value_offset(left_end_idx_),
-        right.value_offset(right_start_idx_), right.values());
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      for (int32_t f = 0; f < num_fields; ++f) {
+        RangeDataEqualsImpl impl(options_, floating_approximate_, *left_.child_data[f],
+                                 *right_.child_data[f],
+                                 left_start_idx_ + left_.offset + i,
+                                 right_start_idx_ + right_.offset + i, length);
+        if (!impl.Compare()) {
+          return false;
+        }
+      }
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const MapArray& left) {
-    result_ = CompareMaps(left);
-    return Status::OK();
-  }
+  Status Visit(const SparseUnionType& type) {
+    const auto& child_ids = type.child_ids();
+    const int8_t* left_codes = left_.GetValues<int8_t>(1);
+    const int8_t* right_codes = right_.GetValues<int8_t>(1);
 
-  Status Visit(const StructArray& left) {
-    result_ = CompareStructs(left);
+    VisitValidRuns([&](int64_t i, int64_t length) {

Review comment:
       Since unions don't have top level nulls is this worthwhile?

##########
File path: cpp/src/arrow/array/diff.h
##########
@@ -59,6 +57,27 @@ ARROW_EXPORT
 Result<std::shared_ptr<StructArray>> Diff(const Array& base, const Array& target,
                                           MemoryPool* pool = default_memory_pool());
 
+/// \brief Compare two array ranges, returning an edit script which expresses the
+/// difference between them
+///
+/// Same as Diff(), but only the ranges defined by the given offsets and lengths
+/// are compared.
+///
+/// \param[in] base baseline for comparison
+/// \param[in] target an array of identical type to base whose elements differ from base's
+/// \param[in] base_offset the start offset of the range to consider inside `base`
+/// \param[in] base_length the length of the range to consider inside `base`
+/// \param[in] target_offset the start offset of the range to consider inside `target`
+/// \param[in] target_length the length of the range to consider inside `target`
+/// \param[in] pool memory to store the result will be allocated from this memory pool
+/// \return an edit script array which can be applied to base to produce target
+ARROW_EXPORT
+Result<std::shared_ptr<StructArray>> DiffRanges(const Array& base, const Array& target,

Review comment:
       Not really an objection, but: I'm not sure what this adds over applying the ranges to the base and target ArrayData then running Diff on those

##########
File path: cpp/src/arrow/compare.cc
##########
@@ -49,700 +51,441 @@
 namespace arrow {
 
 using internal::BitmapEquals;
+using internal::BitmapReader;
+using internal::BitmapUInt64Reader;
 using internal::checked_cast;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitmapEquals;
 
 // ----------------------------------------------------------------------
 // Public method implementations
 
 namespace {
 
-// These helper functions assume we already checked the arrays have equal
-// sizes and null bitmaps.
+bool CompareArrayRanges(const ArrayData& left, const ArrayData& right,
+                        int64_t left_start_idx, int64_t left_end_idx,
+                        int64_t right_start_idx, const EqualOptions& options,
+                        bool floating_approximate);
 
-template <typename ArrowType, typename EqualityFunc>
-inline bool BaseFloatingEquals(const NumericArray<ArrowType>& left,
-                               const NumericArray<ArrowType>& right,
-                               EqualityFunc&& equals) {
-  using T = typename ArrowType::c_type;
-
-  const T* left_data = left.raw_values();
-  const T* right_data = right.raw_values();
-
-  if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i)) continue;
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  } else {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  }
-  return true;
-}
-
-template <typename ArrowType>
-inline bool FloatingEquals(const NumericArray<ArrowType>& left,
-                           const NumericArray<ArrowType>& right,
-                           const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [](T x, T y) -> bool {
-      return (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right,
-                                         [](T x, T y) -> bool { return x == y; });
-  }
-}
-
-template <typename ArrowType>
-inline bool FloatingApproxEquals(const NumericArray<ArrowType>& left,
-                                 const NumericArray<ArrowType>& right,
-                                 const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-  const T epsilon = static_cast<T>(opts.atol());
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y);
-    });
-  }
-}
-
-// RangeEqualsVisitor assumes the range sizes are equal
-
-class RangeEqualsVisitor {
+class RangeDataEqualsImpl {
  public:
-  RangeEqualsVisitor(const Array& right, int64_t left_start_idx, int64_t left_end_idx,
-                     int64_t right_start_idx)
-      : right_(right),
+  // PRE-CONDITIONS:
+  // - the types are equal
+  // - the ranges are in bounds
+  RangeDataEqualsImpl(const EqualOptions& options, bool floating_approximate,
+                      const ArrayData& left, const ArrayData& right,
+                      int64_t left_start_idx, int64_t right_start_idx,
+                      int64_t range_length)
+      : options_(options),
+        floating_approximate_(floating_approximate),
+        left_(left),
+        right_(right),
         left_start_idx_(left_start_idx),
-        left_end_idx_(left_end_idx),
         right_start_idx_(right_start_idx),
+        range_length_(range_length),
         result_(false) {}
 
-  template <typename ArrayType>
-  inline Status CompareValues(const ArrayType& left) {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i) ||
-          (!is_null && left.Value(i) != right.Value(o_i))) {
-        result_ = false;
-        return Status::OK();
+  bool Compare() {
+    // Compare null bitmaps
+    if (left_start_idx_ == 0 && right_start_idx_ == 0 && range_length_ == left_.length &&
+        range_length_ == right_.length) {
+      // If we're comparing entire arrays, we can first compare the cached null counts
+      if (left_.GetNullCount() != right_.GetNullCount()) {
+        return false;
       }
     }
-    result_ = true;
-    return Status::OK();
+    if (!OptionalBitmapEquals(left_.buffers[0], left_.offset + left_start_idx_,
+                              right_.buffers[0], right_.offset + right_start_idx_,
+                              range_length_)) {
+      return false;
+    }
+    // Compare values
+    return CompareWithType(*left_.type);
   }
 
-  template <typename ArrayType, typename CompareValuesFunc>
-  bool CompareWithOffsets(const ArrayType& left,
-                          CompareValuesFunc&& compare_values) const {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        return false;
-      }
-      if (is_null) continue;
-      const auto begin_offset = left.value_offset(i);
-      const auto end_offset = left.value_offset(i + 1);
-      const auto right_begin_offset = right.value_offset(o_i);
-      const auto right_end_offset = right.value_offset(o_i + 1);
-      // Underlying can't be equal if the size isn't equal
-      if (end_offset - begin_offset != right_end_offset - right_begin_offset) {
-        return false;
-      }
-
-      if (!compare_values(left, right, begin_offset, right_begin_offset,
-                          end_offset - begin_offset)) {
-        return false;
-      }
+  bool CompareWithType(const DataType& type) {
+    result_ = true;
+    if (range_length_ != 0) {
+      ARROW_CHECK_OK(VisitTypeInline(type, this));
     }
-    return true;
+    return result_;
   }
 
-  template <typename BinaryArrayType>
-  bool CompareBinaryRange(const BinaryArrayType& left) const {
-    using offset_type = typename BinaryArrayType::offset_type;
+  Status Visit(const NullType&) { return Status::OK(); }
 
-    auto compare_values = [](const BinaryArrayType& left, const BinaryArrayType& right,
-                             offset_type left_offset, offset_type right_offset,
-                             offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return std::memcmp(left.value_data()->data() + left_offset,
-                         right.value_data()->data() + right_offset,
-                         static_cast<size_t>(nvalues)) == 0;
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_primitive_ctype<TypeClass, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  template <typename ListArrayType>
-  bool CompareLists(const ListArrayType& left) {
-    using offset_type = typename ListArrayType::offset_type;
-    const auto& right = checked_cast<const ListArrayType&>(right_);
-    const std::shared_ptr<Array>& left_values = left.values();
-    const std::shared_ptr<Array>& right_values = right.values();
-
-    auto compare_values = [&](const ListArrayType& left, const ListArrayType& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_values->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                      right_values);
-    };
-    return CompareWithOffsets(left, compare_values);
-  }
-
-  bool CompareMaps(const MapArray& left) {
-    // We need a specific comparison helper for maps to avoid comparing
-    // struct field names (which are indifferent for maps)
-    using offset_type = typename MapArray::offset_type;
-    const auto& right = checked_cast<const MapArray&>(right_);
-    const auto left_keys = left.keys();
-    const auto left_items = left.items();
-    const auto right_keys = right.keys();
-    const auto right_items = right.items();
-
-    auto compare_values = [&](const MapArray& left, const MapArray& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_keys->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                    right_keys) &&
-             left_items->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                     right_items);
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_t<is_temporal_type<TypeClass>::value, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  bool CompareStructs(const StructArray& left) {
-    const auto& right = checked_cast<const StructArray&>(right_);
-    bool equal_fields = true;
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      for (int j = 0; j < left.num_fields(); ++j) {
-        // TODO: really we should be comparing stretches of non-null data rather
-        // than looking at one value at a time.
-        equal_fields = left.field(j)->RangeEquals(i, i + 1, o_i, right.field(j));
-        if (!equal_fields) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  bool CompareUnions(const UnionArray& left) const {
-    const auto& right = checked_cast<const UnionArray&>(right_);
-
-    const UnionMode::type union_mode = left.mode();
-    if (union_mode != right.mode()) {
-      return false;
-    }
-
-    const auto& left_type = checked_cast<const UnionType&>(*left.type());
-
-    const std::vector<int>& child_ids = left_type.child_ids();
-
-    const int8_t* left_codes = left.raw_type_codes();
-    const int8_t* right_codes = right.raw_type_codes();
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      if (left_codes[i] != right_codes[o_i]) {
-        return false;
-      }
-
-      auto child_num = child_ids[left_codes[i]];
-
-      // TODO(wesm): really we should be comparing stretches of non-null data
-      // rather than looking at one value at a time.
-      if (union_mode == UnionMode::SPARSE) {
-        if (!left.field(child_num)->RangeEquals(i, i + 1, o_i, right.field(child_num))) {
-          return false;
+  Status Visit(const BooleanType&) {
+    const uint8_t* left_bits = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_bits = right_.GetValues<uint8_t>(1, 0);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      if (length <= 8) {
+        // Avoid the BitmapUInt64Reader overhead for very small runs
+        for (int64_t j = i; j < i + length; ++j) {
+          if (BitUtil::GetBit(left_bits, left_start_idx_ + left_.offset + j) !=
+              BitUtil::GetBit(right_bits, right_start_idx_ + right_.offset + j)) {
+            return false;
+          }
         }
+        return true;
       } else {
-        const int32_t offset =
-            checked_cast<const DenseUnionArray&>(left).raw_value_offsets()[i];
-        const int32_t o_offset =
-            checked_cast<const DenseUnionArray&>(right).raw_value_offsets()[o_i];
-        if (!left.field(child_num)->RangeEquals(offset, offset + 1, o_offset,
-                                                right.field(child_num))) {
-          return false;
+        BitmapUInt64Reader left_reader(left_bits, left_start_idx_ + left_.offset + i,
+                                       length);
+        BitmapUInt64Reader right_reader(right_bits, right_start_idx_ + right_.offset + i,
+                                        length);
+        while (left_reader.position() < length) {
+          if (left_reader.NextWord() != right_reader.NextWord()) {
+            return false;
+          }
         }
+        DCHECK_EQ(right_reader.position(), length);
       }
-    }
-    return true;
-  }
-
-  Status Visit(const BinaryArray& left) {
-    result_ = CompareBinaryRange(left);
-    return Status::OK();
-  }
-
-  Status Visit(const LargeBinaryArray& left) {
-    result_ = CompareBinaryRange(left);
+      return true;
+    };
+    VisitValidRuns(compare_runs);

Review comment:
       This seems like a generally useful bitmap comparison utility. If it's faster than BitmapsEqual then maybe it could replace that function?

##########
File path: cpp/src/arrow/compare.cc
##########
@@ -49,700 +51,441 @@
 namespace arrow {
 
 using internal::BitmapEquals;
+using internal::BitmapReader;
+using internal::BitmapUInt64Reader;
 using internal::checked_cast;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitmapEquals;
 
 // ----------------------------------------------------------------------
 // Public method implementations
 
 namespace {
 
-// These helper functions assume we already checked the arrays have equal
-// sizes and null bitmaps.
+bool CompareArrayRanges(const ArrayData& left, const ArrayData& right,
+                        int64_t left_start_idx, int64_t left_end_idx,
+                        int64_t right_start_idx, const EqualOptions& options,
+                        bool floating_approximate);
 
-template <typename ArrowType, typename EqualityFunc>
-inline bool BaseFloatingEquals(const NumericArray<ArrowType>& left,
-                               const NumericArray<ArrowType>& right,
-                               EqualityFunc&& equals) {
-  using T = typename ArrowType::c_type;
-
-  const T* left_data = left.raw_values();
-  const T* right_data = right.raw_values();
-
-  if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i)) continue;
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  } else {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  }
-  return true;
-}
-
-template <typename ArrowType>
-inline bool FloatingEquals(const NumericArray<ArrowType>& left,
-                           const NumericArray<ArrowType>& right,
-                           const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [](T x, T y) -> bool {
-      return (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right,
-                                         [](T x, T y) -> bool { return x == y; });
-  }
-}
-
-template <typename ArrowType>
-inline bool FloatingApproxEquals(const NumericArray<ArrowType>& left,
-                                 const NumericArray<ArrowType>& right,
-                                 const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-  const T epsilon = static_cast<T>(opts.atol());
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y);
-    });
-  }
-}
-
-// RangeEqualsVisitor assumes the range sizes are equal
-
-class RangeEqualsVisitor {
+class RangeDataEqualsImpl {
  public:
-  RangeEqualsVisitor(const Array& right, int64_t left_start_idx, int64_t left_end_idx,
-                     int64_t right_start_idx)
-      : right_(right),
+  // PRE-CONDITIONS:
+  // - the types are equal
+  // - the ranges are in bounds
+  RangeDataEqualsImpl(const EqualOptions& options, bool floating_approximate,
+                      const ArrayData& left, const ArrayData& right,
+                      int64_t left_start_idx, int64_t right_start_idx,
+                      int64_t range_length)
+      : options_(options),
+        floating_approximate_(floating_approximate),
+        left_(left),
+        right_(right),
         left_start_idx_(left_start_idx),
-        left_end_idx_(left_end_idx),
         right_start_idx_(right_start_idx),
+        range_length_(range_length),
         result_(false) {}
 
-  template <typename ArrayType>
-  inline Status CompareValues(const ArrayType& left) {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i) ||
-          (!is_null && left.Value(i) != right.Value(o_i))) {
-        result_ = false;
-        return Status::OK();
+  bool Compare() {
+    // Compare null bitmaps
+    if (left_start_idx_ == 0 && right_start_idx_ == 0 && range_length_ == left_.length &&
+        range_length_ == right_.length) {
+      // If we're comparing entire arrays, we can first compare the cached null counts
+      if (left_.GetNullCount() != right_.GetNullCount()) {
+        return false;
       }
     }
-    result_ = true;
-    return Status::OK();
+    if (!OptionalBitmapEquals(left_.buffers[0], left_.offset + left_start_idx_,
+                              right_.buffers[0], right_.offset + right_start_idx_,
+                              range_length_)) {
+      return false;
+    }
+    // Compare values
+    return CompareWithType(*left_.type);
   }
 
-  template <typename ArrayType, typename CompareValuesFunc>
-  bool CompareWithOffsets(const ArrayType& left,
-                          CompareValuesFunc&& compare_values) const {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        return false;
-      }
-      if (is_null) continue;
-      const auto begin_offset = left.value_offset(i);
-      const auto end_offset = left.value_offset(i + 1);
-      const auto right_begin_offset = right.value_offset(o_i);
-      const auto right_end_offset = right.value_offset(o_i + 1);
-      // Underlying can't be equal if the size isn't equal
-      if (end_offset - begin_offset != right_end_offset - right_begin_offset) {
-        return false;
-      }
-
-      if (!compare_values(left, right, begin_offset, right_begin_offset,
-                          end_offset - begin_offset)) {
-        return false;
-      }
+  bool CompareWithType(const DataType& type) {
+    result_ = true;
+    if (range_length_ != 0) {
+      ARROW_CHECK_OK(VisitTypeInline(type, this));
     }
-    return true;
+    return result_;
   }
 
-  template <typename BinaryArrayType>
-  bool CompareBinaryRange(const BinaryArrayType& left) const {
-    using offset_type = typename BinaryArrayType::offset_type;
+  Status Visit(const NullType&) { return Status::OK(); }
 
-    auto compare_values = [](const BinaryArrayType& left, const BinaryArrayType& right,
-                             offset_type left_offset, offset_type right_offset,
-                             offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return std::memcmp(left.value_data()->data() + left_offset,
-                         right.value_data()->data() + right_offset,
-                         static_cast<size_t>(nvalues)) == 0;
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_primitive_ctype<TypeClass, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  template <typename ListArrayType>
-  bool CompareLists(const ListArrayType& left) {
-    using offset_type = typename ListArrayType::offset_type;
-    const auto& right = checked_cast<const ListArrayType&>(right_);
-    const std::shared_ptr<Array>& left_values = left.values();
-    const std::shared_ptr<Array>& right_values = right.values();
-
-    auto compare_values = [&](const ListArrayType& left, const ListArrayType& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_values->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                      right_values);
-    };
-    return CompareWithOffsets(left, compare_values);
-  }
-
-  bool CompareMaps(const MapArray& left) {
-    // We need a specific comparison helper for maps to avoid comparing
-    // struct field names (which are indifferent for maps)
-    using offset_type = typename MapArray::offset_type;
-    const auto& right = checked_cast<const MapArray&>(right_);
-    const auto left_keys = left.keys();
-    const auto left_items = left.items();
-    const auto right_keys = right.keys();
-    const auto right_items = right.items();
-
-    auto compare_values = [&](const MapArray& left, const MapArray& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_keys->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                    right_keys) &&
-             left_items->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                     right_items);
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_t<is_temporal_type<TypeClass>::value, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  bool CompareStructs(const StructArray& left) {
-    const auto& right = checked_cast<const StructArray&>(right_);
-    bool equal_fields = true;
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      for (int j = 0; j < left.num_fields(); ++j) {
-        // TODO: really we should be comparing stretches of non-null data rather
-        // than looking at one value at a time.
-        equal_fields = left.field(j)->RangeEquals(i, i + 1, o_i, right.field(j));
-        if (!equal_fields) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  bool CompareUnions(const UnionArray& left) const {
-    const auto& right = checked_cast<const UnionArray&>(right_);
-
-    const UnionMode::type union_mode = left.mode();
-    if (union_mode != right.mode()) {
-      return false;
-    }
-
-    const auto& left_type = checked_cast<const UnionType&>(*left.type());
-
-    const std::vector<int>& child_ids = left_type.child_ids();
-
-    const int8_t* left_codes = left.raw_type_codes();
-    const int8_t* right_codes = right.raw_type_codes();
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      if (left_codes[i] != right_codes[o_i]) {
-        return false;
-      }
-
-      auto child_num = child_ids[left_codes[i]];
-
-      // TODO(wesm): really we should be comparing stretches of non-null data
-      // rather than looking at one value at a time.
-      if (union_mode == UnionMode::SPARSE) {
-        if (!left.field(child_num)->RangeEquals(i, i + 1, o_i, right.field(child_num))) {
-          return false;
+  Status Visit(const BooleanType&) {
+    const uint8_t* left_bits = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_bits = right_.GetValues<uint8_t>(1, 0);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      if (length <= 8) {
+        // Avoid the BitmapUInt64Reader overhead for very small runs
+        for (int64_t j = i; j < i + length; ++j) {
+          if (BitUtil::GetBit(left_bits, left_start_idx_ + left_.offset + j) !=
+              BitUtil::GetBit(right_bits, right_start_idx_ + right_.offset + j)) {
+            return false;
+          }
         }
+        return true;
       } else {
-        const int32_t offset =
-            checked_cast<const DenseUnionArray&>(left).raw_value_offsets()[i];
-        const int32_t o_offset =
-            checked_cast<const DenseUnionArray&>(right).raw_value_offsets()[o_i];
-        if (!left.field(child_num)->RangeEquals(offset, offset + 1, o_offset,
-                                                right.field(child_num))) {
-          return false;
+        BitmapUInt64Reader left_reader(left_bits, left_start_idx_ + left_.offset + i,
+                                       length);
+        BitmapUInt64Reader right_reader(right_bits, right_start_idx_ + right_.offset + i,
+                                        length);
+        while (left_reader.position() < length) {
+          if (left_reader.NextWord() != right_reader.NextWord()) {
+            return false;
+          }
         }
+        DCHECK_EQ(right_reader.position(), length);
       }
-    }
-    return true;
-  }
-
-  Status Visit(const BinaryArray& left) {
-    result_ = CompareBinaryRange(left);
-    return Status::OK();
-  }
-
-  Status Visit(const LargeBinaryArray& left) {
-    result_ = CompareBinaryRange(left);
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const FixedSizeBinaryArray& left) {
-    const auto& right = checked_cast<const FixedSizeBinaryArray&>(right_);
+  Status Visit(const FloatType& type) { return CompareFloating(type); }
 
-    int32_t width = left.byte_width();
+  Status Visit(const DoubleType& type) { return CompareFloating(type); }
 
-    const uint8_t* left_data = nullptr;
-    const uint8_t* right_data = nullptr;
+  // Also matches StringType
+  Status Visit(const BinaryType& type) { return CompareBinary(type); }
 
-    if (left.values()) {
-      left_data = left.raw_values();
-    }
+  // Also matches LargeStringType
+  Status Visit(const LargeBinaryType& type) { return CompareBinary(type); }
 
-    if (right.values()) {
-      right_data = right.raw_values();
-    }
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        result_ = false;
-        return Status::OK();
-      }
-      if (is_null) continue;
+  Status Visit(const FixedSizeBinaryType& type) {
+    const auto byte_width = type.byte_width();
+    const uint8_t* left_data = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_data = right_.GetValues<uint8_t>(1, 0);
 
-      if (std::memcmp(left_data + width * i, right_data + width * o_i, width)) {
-        result_ = false;
-        return Status::OK();
-      }
+    if (left_data != nullptr && right_data != nullptr) {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+        return memcmp(left_data + (left_start_idx_ + left_.offset + i) * byte_width,
+                      right_data + (right_start_idx_ + right_.offset + i) * byte_width,
+                      length * byte_width) == 0;
+      };
+      VisitValidRuns(compare_runs);
+    } else {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool { return true; };
+      VisitValidRuns(compare_runs);
     }
-    result_ = true;
     return Status::OK();
   }
 
-  Status Visit(const Decimal128Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  // Also matches MapType
+  Status Visit(const ListType& type) { return CompareList(type); }
 
-  Status Visit(const Decimal256Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  Status Visit(const LargeListType& type) { return CompareList(type); }
 
-  Status Visit(const NullArray& left) {
-    ARROW_UNUSED(left);
-    result_ = true;
-    return Status::OK();
-  }
-
-  template <typename T>
-  typename std::enable_if<std::is_base_of<PrimitiveArray, T>::value, Status>::type Visit(
-      const T& left) {
-    return CompareValues<T>(left);
-  }
+  Status Visit(const FixedSizeListType& type) {
+    const auto list_size = type.list_size();
+    const ArrayData& left_data = *left_.child_data[0];
+    const ArrayData& right_data = *right_.child_data[0];
 
-  Status Visit(const ListArray& left) {
-    result_ = CompareLists(left);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      RangeDataEqualsImpl impl(options_, floating_approximate_, left_data, right_data,
+                               (left_start_idx_ + left_.offset + i) * list_size,
+                               (right_start_idx_ + right_.offset + i) * list_size,
+                               length * list_size);
+      return impl.Compare();
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const LargeListArray& left) {
-    result_ = CompareLists(left);
-    return Status::OK();
-  }
+  Status Visit(const StructType& type) {
+    const int32_t num_fields = type.num_fields();
 
-  Status Visit(const FixedSizeListArray& left) {
-    const auto& right = checked_cast<const FixedSizeListArray&>(right_);
-    result_ = left.values()->RangeEquals(
-        left.value_offset(left_start_idx_), left.value_offset(left_end_idx_),
-        right.value_offset(right_start_idx_), right.values());
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      for (int32_t f = 0; f < num_fields; ++f) {
+        RangeDataEqualsImpl impl(options_, floating_approximate_, *left_.child_data[f],
+                                 *right_.child_data[f],
+                                 left_start_idx_ + left_.offset + i,
+                                 right_start_idx_ + right_.offset + i, length);
+        if (!impl.Compare()) {
+          return false;
+        }
+      }
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const MapArray& left) {
-    result_ = CompareMaps(left);
-    return Status::OK();
-  }
+  Status Visit(const SparseUnionType& type) {
+    const auto& child_ids = type.child_ids();
+    const int8_t* left_codes = left_.GetValues<int8_t>(1);
+    const int8_t* right_codes = right_.GetValues<int8_t>(1);
 
-  Status Visit(const StructArray& left) {
-    result_ = CompareStructs(left);
+    VisitValidRuns([&](int64_t i, int64_t length) {
+      for (int64_t j = i; j < i + length; ++j) {
+        const auto type_id = left_codes[left_start_idx_ + j];
+        if (type_id != right_codes[right_start_idx_ + j]) {
+          return false;
+        }
+        const auto child_num = child_ids[type_id];
+        // XXX can we instead detect runs of same-child union values?

Review comment:
       Interesting, this seems very doable:
   - get next type_id of left and right
   - early exit if not equal
   - find length of run in left_codes
   - find length of run in right_codes
   - early exit if run lengths are not equal
   - compare ranges of children

##########
File path: cpp/src/arrow/compare.cc
##########
@@ -49,700 +51,441 @@
 namespace arrow {
 
 using internal::BitmapEquals;
+using internal::BitmapReader;
+using internal::BitmapUInt64Reader;
 using internal::checked_cast;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitmapEquals;
 
 // ----------------------------------------------------------------------
 // Public method implementations
 
 namespace {
 
-// These helper functions assume we already checked the arrays have equal
-// sizes and null bitmaps.
+bool CompareArrayRanges(const ArrayData& left, const ArrayData& right,
+                        int64_t left_start_idx, int64_t left_end_idx,
+                        int64_t right_start_idx, const EqualOptions& options,
+                        bool floating_approximate);
 
-template <typename ArrowType, typename EqualityFunc>
-inline bool BaseFloatingEquals(const NumericArray<ArrowType>& left,
-                               const NumericArray<ArrowType>& right,
-                               EqualityFunc&& equals) {
-  using T = typename ArrowType::c_type;
-
-  const T* left_data = left.raw_values();
-  const T* right_data = right.raw_values();
-
-  if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i)) continue;
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  } else {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  }
-  return true;
-}
-
-template <typename ArrowType>
-inline bool FloatingEquals(const NumericArray<ArrowType>& left,
-                           const NumericArray<ArrowType>& right,
-                           const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [](T x, T y) -> bool {
-      return (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right,
-                                         [](T x, T y) -> bool { return x == y; });
-  }
-}
-
-template <typename ArrowType>
-inline bool FloatingApproxEquals(const NumericArray<ArrowType>& left,
-                                 const NumericArray<ArrowType>& right,
-                                 const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-  const T epsilon = static_cast<T>(opts.atol());
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y);
-    });
-  }
-}
-
-// RangeEqualsVisitor assumes the range sizes are equal
-
-class RangeEqualsVisitor {
+class RangeDataEqualsImpl {
  public:
-  RangeEqualsVisitor(const Array& right, int64_t left_start_idx, int64_t left_end_idx,
-                     int64_t right_start_idx)
-      : right_(right),
+  // PRE-CONDITIONS:
+  // - the types are equal
+  // - the ranges are in bounds
+  RangeDataEqualsImpl(const EqualOptions& options, bool floating_approximate,
+                      const ArrayData& left, const ArrayData& right,
+                      int64_t left_start_idx, int64_t right_start_idx,
+                      int64_t range_length)
+      : options_(options),
+        floating_approximate_(floating_approximate),
+        left_(left),
+        right_(right),
         left_start_idx_(left_start_idx),
-        left_end_idx_(left_end_idx),
         right_start_idx_(right_start_idx),
+        range_length_(range_length),
         result_(false) {}
 
-  template <typename ArrayType>
-  inline Status CompareValues(const ArrayType& left) {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i) ||
-          (!is_null && left.Value(i) != right.Value(o_i))) {
-        result_ = false;
-        return Status::OK();
+  bool Compare() {
+    // Compare null bitmaps
+    if (left_start_idx_ == 0 && right_start_idx_ == 0 && range_length_ == left_.length &&
+        range_length_ == right_.length) {
+      // If we're comparing entire arrays, we can first compare the cached null counts
+      if (left_.GetNullCount() != right_.GetNullCount()) {
+        return false;
       }
     }
-    result_ = true;
-    return Status::OK();
+    if (!OptionalBitmapEquals(left_.buffers[0], left_.offset + left_start_idx_,
+                              right_.buffers[0], right_.offset + right_start_idx_,
+                              range_length_)) {
+      return false;
+    }
+    // Compare values
+    return CompareWithType(*left_.type);
   }
 
-  template <typename ArrayType, typename CompareValuesFunc>
-  bool CompareWithOffsets(const ArrayType& left,
-                          CompareValuesFunc&& compare_values) const {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        return false;
-      }
-      if (is_null) continue;
-      const auto begin_offset = left.value_offset(i);
-      const auto end_offset = left.value_offset(i + 1);
-      const auto right_begin_offset = right.value_offset(o_i);
-      const auto right_end_offset = right.value_offset(o_i + 1);
-      // Underlying can't be equal if the size isn't equal
-      if (end_offset - begin_offset != right_end_offset - right_begin_offset) {
-        return false;
-      }
-
-      if (!compare_values(left, right, begin_offset, right_begin_offset,
-                          end_offset - begin_offset)) {
-        return false;
-      }
+  bool CompareWithType(const DataType& type) {
+    result_ = true;
+    if (range_length_ != 0) {
+      ARROW_CHECK_OK(VisitTypeInline(type, this));
     }
-    return true;
+    return result_;
   }
 
-  template <typename BinaryArrayType>
-  bool CompareBinaryRange(const BinaryArrayType& left) const {
-    using offset_type = typename BinaryArrayType::offset_type;
+  Status Visit(const NullType&) { return Status::OK(); }
 
-    auto compare_values = [](const BinaryArrayType& left, const BinaryArrayType& right,
-                             offset_type left_offset, offset_type right_offset,
-                             offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return std::memcmp(left.value_data()->data() + left_offset,
-                         right.value_data()->data() + right_offset,
-                         static_cast<size_t>(nvalues)) == 0;
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_primitive_ctype<TypeClass, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  template <typename ListArrayType>
-  bool CompareLists(const ListArrayType& left) {
-    using offset_type = typename ListArrayType::offset_type;
-    const auto& right = checked_cast<const ListArrayType&>(right_);
-    const std::shared_ptr<Array>& left_values = left.values();
-    const std::shared_ptr<Array>& right_values = right.values();
-
-    auto compare_values = [&](const ListArrayType& left, const ListArrayType& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_values->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                      right_values);
-    };
-    return CompareWithOffsets(left, compare_values);
-  }
-
-  bool CompareMaps(const MapArray& left) {
-    // We need a specific comparison helper for maps to avoid comparing
-    // struct field names (which are indifferent for maps)
-    using offset_type = typename MapArray::offset_type;
-    const auto& right = checked_cast<const MapArray&>(right_);
-    const auto left_keys = left.keys();
-    const auto left_items = left.items();
-    const auto right_keys = right.keys();
-    const auto right_items = right.items();
-
-    auto compare_values = [&](const MapArray& left, const MapArray& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_keys->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                    right_keys) &&
-             left_items->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                     right_items);
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_t<is_temporal_type<TypeClass>::value, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  bool CompareStructs(const StructArray& left) {
-    const auto& right = checked_cast<const StructArray&>(right_);
-    bool equal_fields = true;
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      for (int j = 0; j < left.num_fields(); ++j) {
-        // TODO: really we should be comparing stretches of non-null data rather
-        // than looking at one value at a time.
-        equal_fields = left.field(j)->RangeEquals(i, i + 1, o_i, right.field(j));
-        if (!equal_fields) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  bool CompareUnions(const UnionArray& left) const {
-    const auto& right = checked_cast<const UnionArray&>(right_);
-
-    const UnionMode::type union_mode = left.mode();
-    if (union_mode != right.mode()) {
-      return false;
-    }
-
-    const auto& left_type = checked_cast<const UnionType&>(*left.type());
-
-    const std::vector<int>& child_ids = left_type.child_ids();
-
-    const int8_t* left_codes = left.raw_type_codes();
-    const int8_t* right_codes = right.raw_type_codes();
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      if (left_codes[i] != right_codes[o_i]) {
-        return false;
-      }
-
-      auto child_num = child_ids[left_codes[i]];
-
-      // TODO(wesm): really we should be comparing stretches of non-null data
-      // rather than looking at one value at a time.
-      if (union_mode == UnionMode::SPARSE) {
-        if (!left.field(child_num)->RangeEquals(i, i + 1, o_i, right.field(child_num))) {
-          return false;
+  Status Visit(const BooleanType&) {
+    const uint8_t* left_bits = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_bits = right_.GetValues<uint8_t>(1, 0);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      if (length <= 8) {
+        // Avoid the BitmapUInt64Reader overhead for very small runs
+        for (int64_t j = i; j < i + length; ++j) {
+          if (BitUtil::GetBit(left_bits, left_start_idx_ + left_.offset + j) !=
+              BitUtil::GetBit(right_bits, right_start_idx_ + right_.offset + j)) {
+            return false;
+          }
         }
+        return true;
       } else {
-        const int32_t offset =
-            checked_cast<const DenseUnionArray&>(left).raw_value_offsets()[i];
-        const int32_t o_offset =
-            checked_cast<const DenseUnionArray&>(right).raw_value_offsets()[o_i];
-        if (!left.field(child_num)->RangeEquals(offset, offset + 1, o_offset,
-                                                right.field(child_num))) {
-          return false;
+        BitmapUInt64Reader left_reader(left_bits, left_start_idx_ + left_.offset + i,
+                                       length);
+        BitmapUInt64Reader right_reader(right_bits, right_start_idx_ + right_.offset + i,
+                                        length);
+        while (left_reader.position() < length) {
+          if (left_reader.NextWord() != right_reader.NextWord()) {
+            return false;
+          }
         }
+        DCHECK_EQ(right_reader.position(), length);
       }
-    }
-    return true;
-  }
-
-  Status Visit(const BinaryArray& left) {
-    result_ = CompareBinaryRange(left);
-    return Status::OK();
-  }
-
-  Status Visit(const LargeBinaryArray& left) {
-    result_ = CompareBinaryRange(left);
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const FixedSizeBinaryArray& left) {
-    const auto& right = checked_cast<const FixedSizeBinaryArray&>(right_);
+  Status Visit(const FloatType& type) { return CompareFloating(type); }
 
-    int32_t width = left.byte_width();
+  Status Visit(const DoubleType& type) { return CompareFloating(type); }
 
-    const uint8_t* left_data = nullptr;
-    const uint8_t* right_data = nullptr;
+  // Also matches StringType
+  Status Visit(const BinaryType& type) { return CompareBinary(type); }
 
-    if (left.values()) {
-      left_data = left.raw_values();
-    }
+  // Also matches LargeStringType
+  Status Visit(const LargeBinaryType& type) { return CompareBinary(type); }
 
-    if (right.values()) {
-      right_data = right.raw_values();
-    }
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        result_ = false;
-        return Status::OK();
-      }
-      if (is_null) continue;
+  Status Visit(const FixedSizeBinaryType& type) {
+    const auto byte_width = type.byte_width();
+    const uint8_t* left_data = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_data = right_.GetValues<uint8_t>(1, 0);
 
-      if (std::memcmp(left_data + width * i, right_data + width * o_i, width)) {
-        result_ = false;
-        return Status::OK();
-      }
+    if (left_data != nullptr && right_data != nullptr) {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+        return memcmp(left_data + (left_start_idx_ + left_.offset + i) * byte_width,
+                      right_data + (right_start_idx_ + right_.offset + i) * byte_width,
+                      length * byte_width) == 0;
+      };
+      VisitValidRuns(compare_runs);
+    } else {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool { return true; };
+      VisitValidRuns(compare_runs);
     }
-    result_ = true;
     return Status::OK();
   }
 
-  Status Visit(const Decimal128Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  // Also matches MapType
+  Status Visit(const ListType& type) { return CompareList(type); }
 
-  Status Visit(const Decimal256Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  Status Visit(const LargeListType& type) { return CompareList(type); }
 
-  Status Visit(const NullArray& left) {
-    ARROW_UNUSED(left);
-    result_ = true;
-    return Status::OK();
-  }
-
-  template <typename T>
-  typename std::enable_if<std::is_base_of<PrimitiveArray, T>::value, Status>::type Visit(
-      const T& left) {
-    return CompareValues<T>(left);
-  }
+  Status Visit(const FixedSizeListType& type) {
+    const auto list_size = type.list_size();
+    const ArrayData& left_data = *left_.child_data[0];
+    const ArrayData& right_data = *right_.child_data[0];
 
-  Status Visit(const ListArray& left) {
-    result_ = CompareLists(left);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      RangeDataEqualsImpl impl(options_, floating_approximate_, left_data, right_data,
+                               (left_start_idx_ + left_.offset + i) * list_size,
+                               (right_start_idx_ + right_.offset + i) * list_size,
+                               length * list_size);
+      return impl.Compare();
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const LargeListArray& left) {
-    result_ = CompareLists(left);
-    return Status::OK();
-  }
+  Status Visit(const StructType& type) {
+    const int32_t num_fields = type.num_fields();
 
-  Status Visit(const FixedSizeListArray& left) {
-    const auto& right = checked_cast<const FixedSizeListArray&>(right_);
-    result_ = left.values()->RangeEquals(
-        left.value_offset(left_start_idx_), left.value_offset(left_end_idx_),
-        right.value_offset(right_start_idx_), right.values());
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      for (int32_t f = 0; f < num_fields; ++f) {
+        RangeDataEqualsImpl impl(options_, floating_approximate_, *left_.child_data[f],
+                                 *right_.child_data[f],
+                                 left_start_idx_ + left_.offset + i,
+                                 right_start_idx_ + right_.offset + i, length);
+        if (!impl.Compare()) {
+          return false;
+        }
+      }
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const MapArray& left) {
-    result_ = CompareMaps(left);
-    return Status::OK();
-  }
+  Status Visit(const SparseUnionType& type) {
+    const auto& child_ids = type.child_ids();
+    const int8_t* left_codes = left_.GetValues<int8_t>(1);
+    const int8_t* right_codes = right_.GetValues<int8_t>(1);
 
-  Status Visit(const StructArray& left) {
-    result_ = CompareStructs(left);
+    VisitValidRuns([&](int64_t i, int64_t length) {
+      for (int64_t j = i; j < i + length; ++j) {
+        const auto type_id = left_codes[left_start_idx_ + j];
+        if (type_id != right_codes[right_start_idx_ + j]) {
+          return false;
+        }
+        const auto child_num = child_ids[type_id];
+        // XXX can we instead detect runs of same-child union values?
+        RangeDataEqualsImpl impl(
+            options_, floating_approximate_, *left_.child_data[child_num],
+            *right_.child_data[child_num], left_start_idx_ + left_.offset + j,
+            right_start_idx_ + right_.offset + j, 1);
+        if (!impl.Compare()) {
+          return false;
+        }
+      }
+      return true;
+    });
     return Status::OK();
   }
 
-  Status Visit(const UnionArray& left) {
-    result_ = CompareUnions(left);
+  Status Visit(const DenseUnionType& type) {
+    const auto& child_ids = type.child_ids();
+    const int8_t* left_codes = left_.GetValues<int8_t>(1);
+    const int8_t* right_codes = right_.GetValues<int8_t>(1);
+    const int32_t* left_offsets = left_.GetValues<int32_t>(2);
+    const int32_t* right_offsets = right_.GetValues<int32_t>(2);
+
+    VisitValidRuns([&](int64_t i, int64_t length) {
+      for (int64_t j = i; j < i + length; ++j) {
+        const auto type_id = left_codes[left_start_idx_ + j];
+        if (type_id != right_codes[right_start_idx_ + j]) {
+          return false;
+        }
+        const auto child_num = child_ids[type_id];
+        RangeDataEqualsImpl impl(
+            options_, floating_approximate_, *left_.child_data[child_num],
+            *right_.child_data[child_num], left_offsets[left_start_idx_ + j],
+            right_offsets[right_start_idx_ + j], 1);
+        if (!impl.Compare()) {
+          return false;
+        }
+      }
+      return true;
+    });
     return Status::OK();
   }
 
-  Status Visit(const DictionaryArray& left) {
-    const auto& right = checked_cast<const DictionaryArray&>(right_);
-    if (!left.dictionary()->Equals(right.dictionary())) {
-      result_ = false;
-      return Status::OK();
+  Status Visit(const DictionaryType& type) {
+    // Compare dictionaries
+    result_ &= CompareArrayRanges(
+        *left_.dictionary, *right_.dictionary,
+        /*left_start_idx=*/0,
+        /*left_end_idx=*/std::max(left_.dictionary->length, right_.dictionary->length),
+        /*right_start_idx=*/0, options_, floating_approximate_);
+    if (result_) {
+      // Compare indices
+      result_ &= CompareWithType(*type.index_type());
     }
-    result_ = left.indices()->RangeEquals(left_start_idx_, left_end_idx_,
-                                          right_start_idx_, right.indices());
     return Status::OK();
   }
 
-  Status Visit(const ExtensionArray& left) {
-    result_ = (right_.type()->Equals(*left.type()) &&
-               ArrayRangeEquals(*left.storage(),
-                                *static_cast<const ExtensionArray&>(right_).storage(),
-                                left_start_idx_, left_end_idx_, right_start_idx_));
+  Status Visit(const ExtensionType& type) {
+    // Compare storages
+    result_ &= CompareWithType(*type.storage_type());
     return Status::OK();
   }
 
-  bool result() const { return result_; }
-
  protected:
-  const Array& right_;
-  int64_t left_start_idx_;
-  int64_t left_end_idx_;
-  int64_t right_start_idx_;
-
-  bool result_;
-};
-
-static bool IsEqualPrimitive(const PrimitiveArray& left, const PrimitiveArray& right) {
-  const int byte_width = internal::GetByteWidth(*left.type());
-
-  const uint8_t* left_data = nullptr;
-  const uint8_t* right_data = nullptr;
-
-  if (left.values()) {
-    left_data = left.values()->data() + left.offset() * byte_width;
+  template <typename TypeClass, typename CType = typename TypeClass::c_type>
+  Status ComparePrimitive(const TypeClass&) {
+    const CType* left_values = left_.GetValues<CType>(1);
+    const CType* right_values = right_.GetValues<CType>(1);
+    VisitValidRuns([&](int64_t i, int64_t length) {
+      return memcmp(left_values + left_start_idx_ + i,
+                    right_values + right_start_idx_ + i, length * sizeof(CType)) == 0;
+    });
+    return Status::OK();
   }
 
-  if (right.values()) {
-    right_data = right.values()->data() + right.offset() * byte_width;
-  }
+  template <typename TypeClass>
+  Status CompareFloating(const TypeClass&) {
+    using T = typename TypeClass::c_type;
+    const T* left_values = left_.GetValues<T>(1);
+    const T* right_values = right_.GetValues<T>(1);
 
-  if (byte_width == 0) {
-    // Special case 0-width data, as the data pointers may be null
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i) != right.IsNull(i)) {
-        return false;
-      }
-    }
-    return true;
-  } else if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      const bool left_null = left.IsNull(i);
-      const bool right_null = right.IsNull(i);
-      if (left_null != right_null) {
-        return false;
+    if (floating_approximate_) {
+      const T epsilon = static_cast<T>(options_.atol());
+      if (options_.nans_equal()) {
+        VisitValues([&](int64_t i) {
+          const T x = left_values[i + left_start_idx_];
+          const T y = right_values[i + right_start_idx_];
+          return (fabs(x - y) <= epsilon) || (x == y) || (std::isnan(x) && std::isnan(y));
+        });
+      } else {
+        VisitValues([&](int64_t i) {
+          const T x = left_values[i + left_start_idx_];
+          const T y = right_values[i + right_start_idx_];
+          return (fabs(x - y) <= epsilon) || (x == y);
+        });
       }
-      if (!left_null && memcmp(left_data, right_data, byte_width) != 0) {
-        return false;
+    } else {
+      if (options_.nans_equal()) {
+        VisitValues([&](int64_t i) {
+          const T x = left_values[i + left_start_idx_];
+          const T y = right_values[i + right_start_idx_];
+          return (x == y) || (std::isnan(x) && std::isnan(y));
+        });
+      } else {
+        VisitValues([&](int64_t i) {
+          const T x = left_values[i + left_start_idx_];
+          const T y = right_values[i + right_start_idx_];
+          return x == y;
+        });
       }
-      left_data += byte_width;
-      right_data += byte_width;
     }
-    return true;
-  } else {
-    auto number_of_bytes_to_compare = static_cast<size_t>(byte_width * left.length());
-    return memcmp(left_data, right_data, number_of_bytes_to_compare) == 0;
-  }
-}
-
-// A bit confusing: ArrayEqualsVisitor inherits from RangeEqualsVisitor but
-// doesn't share the same preconditions.
-// When RangeEqualsVisitor is called, we only know the range sizes equal.
-// When ArrayEqualsVisitor is called, we know the sizes and null bitmaps are equal.
-
-class ArrayEqualsVisitor : public RangeEqualsVisitor {
- public:
-  explicit ArrayEqualsVisitor(const Array& right, const EqualOptions& opts)
-      : RangeEqualsVisitor(right, 0, right.length(), 0), opts_(opts) {}
-
-  Status Visit(const NullArray& left) {
-    ARROW_UNUSED(left);
-    result_ = true;
     return Status::OK();
   }
 
-  Status Visit(const BooleanArray& left) {
-    const auto& right = checked_cast<const BooleanArray&>(right_);
+  template <typename TypeClass>
+  Status CompareBinary(const TypeClass&) {
+    const uint8_t* left_data = left_.GetValues<uint8_t>(2, 0);
+    const uint8_t* right_data = right_.GetValues<uint8_t>(2, 0);
 
-    if (left.null_count() > 0) {
-      const uint8_t* left_data = left.values()->data();
-      const uint8_t* right_data = right.values()->data();
-
-      for (int64_t i = 0; i < left.length(); ++i) {
-        if (left.IsValid(i) && BitUtil::GetBit(left_data, i + left.offset()) !=
-                                   BitUtil::GetBit(right_data, i + right.offset())) {
-          result_ = false;
-          return Status::OK();
-        }
-      }
-      result_ = true;
+    if (left_data != nullptr && right_data != nullptr) {
+      const auto compare_ranges = [&](int64_t left_offset, int64_t right_offset,
+                                      int64_t length) -> bool {
+        return memcmp(left_data + left_offset, right_data + right_offset, length) == 0;
+      };
+      CompareWithOffsets<typename TypeClass::offset_type>(1, compare_ranges);
     } else {
-      result_ = BitmapEquals(left.values()->data(), left.offset(), right.values()->data(),
-                             right.offset(), left.length());
+      // One of the arrays is an array of empty strings and nulls.
+      // We just need to compare the offsets.
+      // (note we must not call memcmp() with null data pointers)
+      const auto compare_ranges = [&](int64_t left_offset, int64_t right_offset,
+                                      int64_t length) -> bool { return true; };
+      CompareWithOffsets<typename TypeClass::offset_type>(1, compare_ranges);

Review comment:
       ```suggestion
         CompareWithOffsets<typename TypeClass::offset_type>(1, [](...) { return true; });
   ```

##########
File path: cpp/src/arrow/util/bitmap_reader.h
##########
@@ -69,6 +69,77 @@ class BitmapReader {
   int64_t bit_offset_;
 };
 
+// XXX Cannot name it BitmapWordReader because the name is already used
+// in bitmap_ops.cc

Review comment:
       There's a lot of duplication of logic under the heading of "reading words from bitmaps": VisitWords, BitmapWordReader, BitmapUInt64Reader. In a follow up it'd be handy to consolidate these and ensure they're rigorously benchmarked (or if there are compelling reasons not to consolidate then provide comments indicating when each should be preferred)

##########
File path: cpp/src/arrow/util/bitmap.h
##########
@@ -110,8 +110,8 @@ class ARROW_EXPORT Bitmap : public util::ToStringOstreamable<Bitmap>,
   ///
   /// TODO(bkietz) allow for early termination
   template <size_t N, typename Visitor,
-            typename Word =
-                typename internal::call_traits::argument_type<0, Visitor&&>::value_type>
+            typename Word = typename std::decay<
+                internal::call_traits::argument_type<0, Visitor&&>>::type::value_type>

Review comment:
       I don't think VisitWords is being used in this PR. IIUC this change would only apply if the visitor took a constant reference to the words array rather than just taking the words array?

##########
File path: cpp/src/arrow/compare.cc
##########
@@ -49,700 +51,441 @@
 namespace arrow {
 
 using internal::BitmapEquals;
+using internal::BitmapReader;
+using internal::BitmapUInt64Reader;
 using internal::checked_cast;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitmapEquals;
 
 // ----------------------------------------------------------------------
 // Public method implementations
 
 namespace {
 
-// These helper functions assume we already checked the arrays have equal
-// sizes and null bitmaps.
+bool CompareArrayRanges(const ArrayData& left, const ArrayData& right,
+                        int64_t left_start_idx, int64_t left_end_idx,
+                        int64_t right_start_idx, const EqualOptions& options,
+                        bool floating_approximate);
 
-template <typename ArrowType, typename EqualityFunc>
-inline bool BaseFloatingEquals(const NumericArray<ArrowType>& left,
-                               const NumericArray<ArrowType>& right,
-                               EqualityFunc&& equals) {
-  using T = typename ArrowType::c_type;
-
-  const T* left_data = left.raw_values();
-  const T* right_data = right.raw_values();
-
-  if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i)) continue;
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  } else {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (!equals(left_data[i], right_data[i])) {
-        return false;
-      }
-    }
-  }
-  return true;
-}
-
-template <typename ArrowType>
-inline bool FloatingEquals(const NumericArray<ArrowType>& left,
-                           const NumericArray<ArrowType>& right,
-                           const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [](T x, T y) -> bool {
-      return (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right,
-                                         [](T x, T y) -> bool { return x == y; });
-  }
-}
-
-template <typename ArrowType>
-inline bool FloatingApproxEquals(const NumericArray<ArrowType>& left,
-                                 const NumericArray<ArrowType>& right,
-                                 const EqualOptions& opts) {
-  using T = typename ArrowType::c_type;
-  const T epsilon = static_cast<T>(opts.atol());
-
-  if (opts.nans_equal()) {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y) || (std::isnan(x) && std::isnan(y));
-    });
-  } else {
-    return BaseFloatingEquals<ArrowType>(left, right, [epsilon](T x, T y) -> bool {
-      return (fabs(x - y) <= epsilon) || (x == y);
-    });
-  }
-}
-
-// RangeEqualsVisitor assumes the range sizes are equal
-
-class RangeEqualsVisitor {
+class RangeDataEqualsImpl {
  public:
-  RangeEqualsVisitor(const Array& right, int64_t left_start_idx, int64_t left_end_idx,
-                     int64_t right_start_idx)
-      : right_(right),
+  // PRE-CONDITIONS:
+  // - the types are equal
+  // - the ranges are in bounds
+  RangeDataEqualsImpl(const EqualOptions& options, bool floating_approximate,
+                      const ArrayData& left, const ArrayData& right,
+                      int64_t left_start_idx, int64_t right_start_idx,
+                      int64_t range_length)
+      : options_(options),
+        floating_approximate_(floating_approximate),
+        left_(left),
+        right_(right),
         left_start_idx_(left_start_idx),
-        left_end_idx_(left_end_idx),
         right_start_idx_(right_start_idx),
+        range_length_(range_length),
         result_(false) {}
 
-  template <typename ArrayType>
-  inline Status CompareValues(const ArrayType& left) {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i) ||
-          (!is_null && left.Value(i) != right.Value(o_i))) {
-        result_ = false;
-        return Status::OK();
+  bool Compare() {
+    // Compare null bitmaps
+    if (left_start_idx_ == 0 && right_start_idx_ == 0 && range_length_ == left_.length &&
+        range_length_ == right_.length) {
+      // If we're comparing entire arrays, we can first compare the cached null counts
+      if (left_.GetNullCount() != right_.GetNullCount()) {
+        return false;
       }
     }
-    result_ = true;
-    return Status::OK();
+    if (!OptionalBitmapEquals(left_.buffers[0], left_.offset + left_start_idx_,
+                              right_.buffers[0], right_.offset + right_start_idx_,
+                              range_length_)) {
+      return false;
+    }
+    // Compare values
+    return CompareWithType(*left_.type);
   }
 
-  template <typename ArrayType, typename CompareValuesFunc>
-  bool CompareWithOffsets(const ArrayType& left,
-                          CompareValuesFunc&& compare_values) const {
-    const auto& right = checked_cast<const ArrayType&>(right_);
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        return false;
-      }
-      if (is_null) continue;
-      const auto begin_offset = left.value_offset(i);
-      const auto end_offset = left.value_offset(i + 1);
-      const auto right_begin_offset = right.value_offset(o_i);
-      const auto right_end_offset = right.value_offset(o_i + 1);
-      // Underlying can't be equal if the size isn't equal
-      if (end_offset - begin_offset != right_end_offset - right_begin_offset) {
-        return false;
-      }
-
-      if (!compare_values(left, right, begin_offset, right_begin_offset,
-                          end_offset - begin_offset)) {
-        return false;
-      }
+  bool CompareWithType(const DataType& type) {
+    result_ = true;
+    if (range_length_ != 0) {
+      ARROW_CHECK_OK(VisitTypeInline(type, this));
     }
-    return true;
+    return result_;
   }
 
-  template <typename BinaryArrayType>
-  bool CompareBinaryRange(const BinaryArrayType& left) const {
-    using offset_type = typename BinaryArrayType::offset_type;
+  Status Visit(const NullType&) { return Status::OK(); }
 
-    auto compare_values = [](const BinaryArrayType& left, const BinaryArrayType& right,
-                             offset_type left_offset, offset_type right_offset,
-                             offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return std::memcmp(left.value_data()->data() + left_offset,
-                         right.value_data()->data() + right_offset,
-                         static_cast<size_t>(nvalues)) == 0;
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_primitive_ctype<TypeClass, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  template <typename ListArrayType>
-  bool CompareLists(const ListArrayType& left) {
-    using offset_type = typename ListArrayType::offset_type;
-    const auto& right = checked_cast<const ListArrayType&>(right_);
-    const std::shared_ptr<Array>& left_values = left.values();
-    const std::shared_ptr<Array>& right_values = right.values();
-
-    auto compare_values = [&](const ListArrayType& left, const ListArrayType& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_values->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                      right_values);
-    };
-    return CompareWithOffsets(left, compare_values);
-  }
-
-  bool CompareMaps(const MapArray& left) {
-    // We need a specific comparison helper for maps to avoid comparing
-    // struct field names (which are indifferent for maps)
-    using offset_type = typename MapArray::offset_type;
-    const auto& right = checked_cast<const MapArray&>(right_);
-    const auto left_keys = left.keys();
-    const auto left_items = left.items();
-    const auto right_keys = right.keys();
-    const auto right_items = right.items();
-
-    auto compare_values = [&](const MapArray& left, const MapArray& right,
-                              offset_type left_offset, offset_type right_offset,
-                              offset_type nvalues) {
-      if (nvalues == 0) {
-        return true;
-      }
-      return left_keys->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                    right_keys) &&
-             left_items->RangeEquals(left_offset, left_offset + nvalues, right_offset,
-                                     right_items);
-    };
-    return CompareWithOffsets(left, compare_values);
+  template <typename TypeClass>
+  enable_if_t<is_temporal_type<TypeClass>::value, Status> Visit(const TypeClass& type) {
+    return ComparePrimitive(type);
   }
 
-  bool CompareStructs(const StructArray& left) {
-    const auto& right = checked_cast<const StructArray&>(right_);
-    bool equal_fields = true;
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      for (int j = 0; j < left.num_fields(); ++j) {
-        // TODO: really we should be comparing stretches of non-null data rather
-        // than looking at one value at a time.
-        equal_fields = left.field(j)->RangeEquals(i, i + 1, o_i, right.field(j));
-        if (!equal_fields) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  bool CompareUnions(const UnionArray& left) const {
-    const auto& right = checked_cast<const UnionArray&>(right_);
-
-    const UnionMode::type union_mode = left.mode();
-    if (union_mode != right.mode()) {
-      return false;
-    }
-
-    const auto& left_type = checked_cast<const UnionType&>(*left.type());
-
-    const std::vector<int>& child_ids = left_type.child_ids();
-
-    const int8_t* left_codes = left.raw_type_codes();
-    const int8_t* right_codes = right.raw_type_codes();
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      if (left.IsNull(i) != right.IsNull(o_i)) {
-        return false;
-      }
-      if (left.IsNull(i)) continue;
-      if (left_codes[i] != right_codes[o_i]) {
-        return false;
-      }
-
-      auto child_num = child_ids[left_codes[i]];
-
-      // TODO(wesm): really we should be comparing stretches of non-null data
-      // rather than looking at one value at a time.
-      if (union_mode == UnionMode::SPARSE) {
-        if (!left.field(child_num)->RangeEquals(i, i + 1, o_i, right.field(child_num))) {
-          return false;
+  Status Visit(const BooleanType&) {
+    const uint8_t* left_bits = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_bits = right_.GetValues<uint8_t>(1, 0);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      if (length <= 8) {
+        // Avoid the BitmapUInt64Reader overhead for very small runs
+        for (int64_t j = i; j < i + length; ++j) {
+          if (BitUtil::GetBit(left_bits, left_start_idx_ + left_.offset + j) !=
+              BitUtil::GetBit(right_bits, right_start_idx_ + right_.offset + j)) {
+            return false;
+          }
         }
+        return true;
       } else {
-        const int32_t offset =
-            checked_cast<const DenseUnionArray&>(left).raw_value_offsets()[i];
-        const int32_t o_offset =
-            checked_cast<const DenseUnionArray&>(right).raw_value_offsets()[o_i];
-        if (!left.field(child_num)->RangeEquals(offset, offset + 1, o_offset,
-                                                right.field(child_num))) {
-          return false;
+        BitmapUInt64Reader left_reader(left_bits, left_start_idx_ + left_.offset + i,
+                                       length);
+        BitmapUInt64Reader right_reader(right_bits, right_start_idx_ + right_.offset + i,
+                                        length);
+        while (left_reader.position() < length) {
+          if (left_reader.NextWord() != right_reader.NextWord()) {
+            return false;
+          }
         }
+        DCHECK_EQ(right_reader.position(), length);
       }
-    }
-    return true;
-  }
-
-  Status Visit(const BinaryArray& left) {
-    result_ = CompareBinaryRange(left);
-    return Status::OK();
-  }
-
-  Status Visit(const LargeBinaryArray& left) {
-    result_ = CompareBinaryRange(left);
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const FixedSizeBinaryArray& left) {
-    const auto& right = checked_cast<const FixedSizeBinaryArray&>(right_);
+  Status Visit(const FloatType& type) { return CompareFloating(type); }
 
-    int32_t width = left.byte_width();
+  Status Visit(const DoubleType& type) { return CompareFloating(type); }
 
-    const uint8_t* left_data = nullptr;
-    const uint8_t* right_data = nullptr;
+  // Also matches StringType
+  Status Visit(const BinaryType& type) { return CompareBinary(type); }
 
-    if (left.values()) {
-      left_data = left.raw_values();
-    }
+  // Also matches LargeStringType
+  Status Visit(const LargeBinaryType& type) { return CompareBinary(type); }
 
-    if (right.values()) {
-      right_data = right.raw_values();
-    }
-
-    for (int64_t i = left_start_idx_, o_i = right_start_idx_; i < left_end_idx_;
-         ++i, ++o_i) {
-      const bool is_null = left.IsNull(i);
-      if (is_null != right.IsNull(o_i)) {
-        result_ = false;
-        return Status::OK();
-      }
-      if (is_null) continue;
+  Status Visit(const FixedSizeBinaryType& type) {
+    const auto byte_width = type.byte_width();
+    const uint8_t* left_data = left_.GetValues<uint8_t>(1, 0);
+    const uint8_t* right_data = right_.GetValues<uint8_t>(1, 0);
 
-      if (std::memcmp(left_data + width * i, right_data + width * o_i, width)) {
-        result_ = false;
-        return Status::OK();
-      }
+    if (left_data != nullptr && right_data != nullptr) {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+        return memcmp(left_data + (left_start_idx_ + left_.offset + i) * byte_width,
+                      right_data + (right_start_idx_ + right_.offset + i) * byte_width,
+                      length * byte_width) == 0;
+      };
+      VisitValidRuns(compare_runs);
+    } else {
+      auto compare_runs = [&](int64_t i, int64_t length) -> bool { return true; };
+      VisitValidRuns(compare_runs);
     }
-    result_ = true;
     return Status::OK();
   }
 
-  Status Visit(const Decimal128Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  // Also matches MapType
+  Status Visit(const ListType& type) { return CompareList(type); }
 
-  Status Visit(const Decimal256Array& left) {
-    return Visit(checked_cast<const FixedSizeBinaryArray&>(left));
-  }
+  Status Visit(const LargeListType& type) { return CompareList(type); }
 
-  Status Visit(const NullArray& left) {
-    ARROW_UNUSED(left);
-    result_ = true;
-    return Status::OK();
-  }
-
-  template <typename T>
-  typename std::enable_if<std::is_base_of<PrimitiveArray, T>::value, Status>::type Visit(
-      const T& left) {
-    return CompareValues<T>(left);
-  }
+  Status Visit(const FixedSizeListType& type) {
+    const auto list_size = type.list_size();
+    const ArrayData& left_data = *left_.child_data[0];
+    const ArrayData& right_data = *right_.child_data[0];
 
-  Status Visit(const ListArray& left) {
-    result_ = CompareLists(left);
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      RangeDataEqualsImpl impl(options_, floating_approximate_, left_data, right_data,
+                               (left_start_idx_ + left_.offset + i) * list_size,
+                               (right_start_idx_ + right_.offset + i) * list_size,
+                               length * list_size);
+      return impl.Compare();
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const LargeListArray& left) {
-    result_ = CompareLists(left);
-    return Status::OK();
-  }
+  Status Visit(const StructType& type) {
+    const int32_t num_fields = type.num_fields();
 
-  Status Visit(const FixedSizeListArray& left) {
-    const auto& right = checked_cast<const FixedSizeListArray&>(right_);
-    result_ = left.values()->RangeEquals(
-        left.value_offset(left_start_idx_), left.value_offset(left_end_idx_),
-        right.value_offset(right_start_idx_), right.values());
+    auto compare_runs = [&](int64_t i, int64_t length) -> bool {
+      for (int32_t f = 0; f < num_fields; ++f) {
+        RangeDataEqualsImpl impl(options_, floating_approximate_, *left_.child_data[f],
+                                 *right_.child_data[f],
+                                 left_start_idx_ + left_.offset + i,
+                                 right_start_idx_ + right_.offset + i, length);
+        if (!impl.Compare()) {
+          return false;
+        }
+      }
+      return true;
+    };
+    VisitValidRuns(compare_runs);
     return Status::OK();
   }
 
-  Status Visit(const MapArray& left) {
-    result_ = CompareMaps(left);
-    return Status::OK();
-  }
+  Status Visit(const SparseUnionType& type) {
+    const auto& child_ids = type.child_ids();
+    const int8_t* left_codes = left_.GetValues<int8_t>(1);
+    const int8_t* right_codes = right_.GetValues<int8_t>(1);
 
-  Status Visit(const StructArray& left) {
-    result_ = CompareStructs(left);
+    VisitValidRuns([&](int64_t i, int64_t length) {
+      for (int64_t j = i; j < i + length; ++j) {
+        const auto type_id = left_codes[left_start_idx_ + j];
+        if (type_id != right_codes[right_start_idx_ + j]) {
+          return false;
+        }
+        const auto child_num = child_ids[type_id];
+        // XXX can we instead detect runs of same-child union values?
+        RangeDataEqualsImpl impl(
+            options_, floating_approximate_, *left_.child_data[child_num],
+            *right_.child_data[child_num], left_start_idx_ + left_.offset + j,
+            right_start_idx_ + right_.offset + j, 1);
+        if (!impl.Compare()) {
+          return false;
+        }
+      }
+      return true;
+    });
     return Status::OK();
   }
 
-  Status Visit(const UnionArray& left) {
-    result_ = CompareUnions(left);
+  Status Visit(const DenseUnionType& type) {
+    const auto& child_ids = type.child_ids();
+    const int8_t* left_codes = left_.GetValues<int8_t>(1);
+    const int8_t* right_codes = right_.GetValues<int8_t>(1);
+    const int32_t* left_offsets = left_.GetValues<int32_t>(2);
+    const int32_t* right_offsets = right_.GetValues<int32_t>(2);
+
+    VisitValidRuns([&](int64_t i, int64_t length) {
+      for (int64_t j = i; j < i + length; ++j) {
+        const auto type_id = left_codes[left_start_idx_ + j];
+        if (type_id != right_codes[right_start_idx_ + j]) {
+          return false;
+        }
+        const auto child_num = child_ids[type_id];

Review comment:
       runs of type_id can still be used here, but ranges of children can't be directly compared due to the offsets. It's odd that we "dereference" offsets when comparing dense union arrays but we don't do the same with a dictionary array's indices




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

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