You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ko...@apache.org on 2019/08/13 21:31:41 UTC

[arrow] branch master updated: ARROW-6177: [C++] Add Array::Validate()

This is an automated email from the ASF dual-hosted git repository.

kou pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 13851d4  ARROW-6177: [C++] Add Array::Validate()
13851d4 is described below

commit 13851d40e4b9f85b17c2963b4886e128799d861c
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Wed Aug 14 06:31:15 2019 +0900

    ARROW-6177: [C++] Add Array::Validate()
    
    This is in line with ChunkedArray::Validate() and Table::Validate().
    
    Closes #5067 from pitrou/ARROW-6177-array-validate and squashes the following commits:
    
    52244166e <Antoine Pitrou> ARROW-6177:  Add Arrow::Validate()
    
    Authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Sutou Kouhei <ko...@clear-code.com>
---
 cpp/src/arrow/array-binary-test.cc                |  6 +-
 cpp/src/arrow/array-dict-test.cc                  |  4 +-
 cpp/src/arrow/array-list-test.cc                  | 30 +++++-----
 cpp/src/arrow/array-struct-test.cc                |  8 +--
 cpp/src/arrow/array-view-test.cc                  | 14 ++---
 cpp/src/arrow/array.cc                            | 27 +++++----
 cpp/src/arrow/array.h                             | 14 +++--
 cpp/src/arrow/array/concatenate-test.cc           |  8 +--
 cpp/src/arrow/compute/kernels/argsort-test.cc     |  2 +-
 cpp/src/arrow/compute/kernels/cast-test.cc        |  8 +--
 cpp/src/arrow/compute/kernels/filter-test.cc      | 12 ++--
 cpp/src/arrow/compute/kernels/take-test.cc        |  8 +--
 cpp/src/arrow/flight/perf-server.cc               |  2 +-
 cpp/src/arrow/ipc/json-simple-test.cc             | 70 +++++++++++------------
 cpp/src/arrow/ipc/test-common.cc                  |  2 +-
 cpp/src/parquet/arrow/arrow-reader-writer-test.cc |  1 +
 python/pyarrow/array.pxi                          |  6 +-
 python/pyarrow/includes/libarrow.pxd              |  4 +-
 18 files changed, 119 insertions(+), 107 deletions(-)

diff --git a/cpp/src/arrow/array-binary-test.cc b/cpp/src/arrow/array-binary-test.cc
index ed53895..9dd6c48 100644
--- a/cpp/src/arrow/array-binary-test.cc
+++ b/cpp/src/arrow/array-binary-test.cc
@@ -101,7 +101,7 @@ class TestStringArray : public ::testing::Test {
   void TestArrayBasics() {
     ASSERT_EQ(length_, strings_->length());
     ASSERT_EQ(1, strings_->null_count());
-    ASSERT_OK(ValidateArray(*strings_));
+    ASSERT_OK(strings_->Validate());
     TestInitialized(*strings_);
     AssertZeroPadded(*strings_);
   }
@@ -276,7 +276,7 @@ class TestStringBuilder : public TestBuilder {
     FinishAndCheckPadding(builder_.get(), &out);
 
     result_ = std::dynamic_pointer_cast<ArrayType>(out);
-    ASSERT_OK(ValidateArray(*result_));
+    ASSERT_OK(result_->Validate());
   }
 
   void TestScalarAppend() {
@@ -330,7 +330,7 @@ class TestStringBuilder : public TestBuilder {
     ASSERT_EQ(builder_->value_data_length(), total_length * reps);
     Done();
 
-    ASSERT_OK(ValidateArray(*result_));
+    ASSERT_OK(result_->Validate());
     ASSERT_EQ(reps * N, result_->length());
     ASSERT_EQ(reps, result_->null_count());
     ASSERT_EQ(reps * total_length, result_->value_data()->size());
diff --git a/cpp/src/arrow/array-dict-test.cc b/cpp/src/arrow/array-dict-test.cc
index d1f7739..ca6552c 100644
--- a/cpp/src/arrow/array-dict-test.cc
+++ b/cpp/src/arrow/array-dict-test.cc
@@ -914,7 +914,7 @@ TEST(TestDictionary, Validate) {
       std::make_shared<DictionaryArray>(dict_type, indices, dict);
 
   // Only checking index type for now
-  ASSERT_OK(ValidateArray(*arr));
+  ASSERT_OK(arr->Validate());
 
   ASSERT_DEATH(
       {
@@ -1042,7 +1042,7 @@ TEST(TestDictionary, DISABLED_ListOfDictionary) {
 
   std::shared_ptr<Array> array;
   ASSERT_OK(root_builder->Finish(&array));
-  ASSERT_OK(ValidateArray(*array));
+  ASSERT_OK(array->Validate());
 
   auto expected_type = list(dictionary(int16(), utf8()));
   ASSERT_EQ(array->type()->ToString(), expected_type->ToString());
diff --git a/cpp/src/arrow/array-list-test.cc b/cpp/src/arrow/array-list-test.cc
index 9adaf04..9c802d4 100644
--- a/cpp/src/arrow/array-list-test.cc
+++ b/cpp/src/arrow/array-list-test.cc
@@ -72,7 +72,7 @@ class TestListArray : public TestBuilder {
 
   void ValidateBasicListArray(const ArrayType* result, const std::vector<int16_t>& values,
                               const std::vector<uint8_t>& is_valid) {
-    ASSERT_OK(ValidateArray(*result));
+    ASSERT_OK(result->Validate());
     ASSERT_EQ(1, result->null_count());
     ASSERT_EQ(0, result->values()->null_count());
 
@@ -213,9 +213,9 @@ class TestListArray : public TestBuilder {
     ASSERT_OK(ArrayType::FromArrays(*offsets1, *values, pool_, &list1));
     ASSERT_OK(ArrayType::FromArrays(*offsets3, *values, pool_, &list3));
     ASSERT_OK(ArrayType::FromArrays(*offsets4, *values, pool_, &list4));
-    ASSERT_OK(ValidateArray(*list1));
-    ASSERT_OK(ValidateArray(*list3));
-    ASSERT_OK(ValidateArray(*list4));
+    ASSERT_OK(list1->Validate());
+    ASSERT_OK(list3->Validate());
+    ASSERT_OK(list4->Validate());
 
     ArrayType expected1(list_type, length, offsets1->data()->buffers[1], values,
                         offsets1->data()->buffers[0], 0);
@@ -251,7 +251,7 @@ class TestListArray : public TestBuilder {
 
     Done();
 
-    ASSERT_OK(ValidateArray(*result_));
+    ASSERT_OK(result_->Validate());
     ASSERT_TRUE(result_->IsNull(0));
     ASSERT_TRUE(result_->IsNull(1));
 
@@ -270,7 +270,7 @@ class TestListArray : public TestBuilder {
 
     Done();
 
-    ASSERT_OK(ValidateArray(*result_));
+    ASSERT_OK(result_->Validate());
     ASSERT_EQ(result_->length(), 3);
     ASSERT_EQ(result_->null_count(), 3);
     ASSERT_TRUE(result_->IsNull(0));
@@ -321,13 +321,13 @@ class TestListArray : public TestBuilder {
     }
 
     Done();
-    ASSERT_RAISES(Invalid, ValidateArray(*result_));
+    ASSERT_RAISES(Invalid, result_->Validate());
   }
 
   void TestZeroLength() {
     // All buffers are null
     Done();
-    ASSERT_OK(ValidateArray(*result_));
+    ASSERT_OK(result_->Validate());
   }
 
   void TestBuilderPreserveFieldName() {
@@ -485,7 +485,7 @@ TEST_F(TestMapArray, BuildingIntToInt) {
   ASSERT_OK(item_builder->AppendValues({-1, -1, 0, 1, -1, 2}, {0, 0, 1, 1, 0, 1}));
   ASSERT_OK(map_builder.Append());
   ASSERT_OK(map_builder.Finish(&actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
 
   ASSERT_ARRAYS_EQUAL(*actual, expected);
 }
@@ -518,7 +518,7 @@ TEST_F(TestMapArray, BuildingStringToInt) {
   ASSERT_OK(item_builder->Append(8));
   ASSERT_OK(map_builder.Append());
   ASSERT_OK(map_builder.Finish(&actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
 
   ASSERT_ARRAYS_EQUAL(*actual, expected);
 }
@@ -594,7 +594,7 @@ TEST_F(TestFixedSizeListArray, TestAppendNull) {
 
   Done();
 
-  ASSERT_OK(ValidateArray(*result_));
+  ASSERT_OK(result_->Validate());
   ASSERT_TRUE(result_->IsNull(0));
   ASSERT_TRUE(result_->IsNull(1));
 
@@ -610,7 +610,7 @@ TEST_F(TestFixedSizeListArray, TestAppendNulls) {
 
   Done();
 
-  ASSERT_OK(ValidateArray(*result_));
+  ASSERT_OK(result_->Validate());
   ASSERT_EQ(result_->length(), 3);
   ASSERT_EQ(result_->null_count(), 3);
   ASSERT_TRUE(result_->IsNull(0));
@@ -628,7 +628,7 @@ TEST_F(TestFixedSizeListArray, TestAppendNulls) {
 void ValidateBasicFixedSizeListArray(const FixedSizeListArray* result,
                                      const std::vector<int32_t>& values,
                                      const std::vector<uint8_t>& is_valid) {
-  ASSERT_OK(ValidateArray(*result));
+  ASSERT_OK(result->Validate());
   ASSERT_EQ(1, result->null_count());
   ASSERT_LE(result->values()->null_count(), 2);
 
@@ -704,13 +704,13 @@ TEST_F(TestFixedSizeListArray, BulkAppendInvalid) {
   }
 
   Done();
-  ASSERT_RAISES(Invalid, ValidateArray(*result_));
+  ASSERT_RAISES(Invalid, result_->Validate());
 }
 
 TEST_F(TestFixedSizeListArray, TestZeroLength) {
   // All buffers are null
   Done();
-  ASSERT_OK(ValidateArray(*result_));
+  ASSERT_OK(result_->Validate());
 }
 
 TEST_F(TestFixedSizeListArray, TestBuilderPreserveFieleName) {
diff --git a/cpp/src/arrow/array-struct-test.cc b/cpp/src/arrow/array-struct-test.cc
index 9acd2f9..cfeb416 100644
--- a/cpp/src/arrow/array-struct-test.cc
+++ b/cpp/src/arrow/array-struct-test.cc
@@ -45,7 +45,7 @@ void ValidateBasicStructArray(const StructArray* result,
                               const std::vector<int>& list_offsets,
                               const std::vector<int32_t>& int_values) {
   ASSERT_EQ(4, result->length());
-  ASSERT_OK(ValidateArray(*result));
+  ASSERT_OK(result->Validate());
 
   auto list_char_arr = std::dynamic_pointer_cast<ListArray>(result->field(0));
   auto char_arr = std::dynamic_pointer_cast<Int8Array>(list_char_arr->values());
@@ -225,7 +225,7 @@ TEST_F(TestStructBuilder, TestAppendNull) {
 
   Done();
 
-  ASSERT_OK(ValidateArray(*result_));
+  ASSERT_OK(result_->Validate());
 
   ASSERT_EQ(2, static_cast<int>(result_->num_fields()));
   ASSERT_EQ(2, result_->length());
@@ -339,7 +339,7 @@ TEST_F(TestStructBuilder, BulkAppendInvalid) {
 
   Done();
   // Even null bitmap of the parent Struct is not valid, validate will ignore it.
-  ASSERT_OK(ValidateArray(*result_));
+  ASSERT_OK(result_->Validate());
 }
 
 TEST_F(TestStructBuilder, TestEquality) {
@@ -475,7 +475,7 @@ TEST_F(TestStructBuilder, TestEquality) {
 TEST_F(TestStructBuilder, TestZeroLength) {
   // All buffers are null
   Done();
-  ASSERT_OK(ValidateArray(*result_));
+  ASSERT_OK(result_->Validate());
 }
 
 TEST_F(TestStructBuilder, TestSlice) {
diff --git a/cpp/src/arrow/array-view-test.cc b/cpp/src/arrow/array-view-test.cc
index f466c1d..8d94242 100644
--- a/cpp/src/arrow/array-view-test.cc
+++ b/cpp/src/arrow/array-view-test.cc
@@ -33,7 +33,7 @@ void CheckView(const std::shared_ptr<Array>& input,
                const std::shared_ptr<Array>& expected) {
   std::shared_ptr<Array> result;
   ASSERT_OK(input->View(view_type, &result));
-  ASSERT_OK(ValidateArray(*result));
+  ASSERT_OK(result->Validate());
   AssertArraysEqual(*expected, *result);
 }
 
@@ -316,7 +316,7 @@ TEST(TestArrayView, SparseUnionAsStruct) {
   auto indices = ArrayFromJSON(int8(), "[0, 0, 1]");
   std::shared_ptr<Array> arr;
   ASSERT_OK(UnionArray::MakeSparse(*indices, {child1, child2}, &arr));
-  ASSERT_OK(ValidateArray(*arr));
+  ASSERT_OK(arr->Validate());
 
   auto ty1 = struct_({field("a", int8()), field("b", uint16()), field("c", float32())});
   auto expected = ArrayFromJSON(ty1, "[[0, 0, 0], [0, 65535, 1.5], [1, 42, -2.5]]");
@@ -326,7 +326,7 @@ TEST(TestArrayView, SparseUnionAsStruct) {
   // With nulls
   indices = ArrayFromJSON(int8(), "[null, 0, 1]");
   ASSERT_OK(UnionArray::MakeSparse(*indices, {child1, child2}, &arr));
-  ASSERT_OK(ValidateArray(*arr));
+  ASSERT_OK(arr->Validate());
   expected = ArrayFromJSON(ty1, "[null, [0, 65535, 1.5], [1, 42, -2.5]]");
   CheckView(arr, expected);
   //   CheckView(expected, arr);  // XXX currently fails
@@ -335,7 +335,7 @@ TEST(TestArrayView, SparseUnionAsStruct) {
   child1 = ArrayFromJSON(int16(), "[0, -1, null]");
   child2 = ArrayFromJSON(int32(), "[0, null, -1071644672]");
   ASSERT_OK(UnionArray::MakeSparse(*indices, {child1, child2}, &arr));
-  ASSERT_OK(ValidateArray(*arr));
+  ASSERT_OK(arr->Validate());
   expected = ArrayFromJSON(ty1, "[null, [0, 65535, null], [1, null, -2.5]]");
   CheckView(arr, expected);
   //   CheckView(expected, arr);  // XXX currently fails
@@ -348,9 +348,9 @@ TEST(TestArrayView, DecimalRoundTrip) {
   auto ty2 = fixed_size_binary(16);
   std::shared_ptr<Array> v, w;
   ASSERT_OK(arr->View(ty2, &v));
-  ASSERT_OK(ValidateArray(*v));
+  ASSERT_OK(v->Validate());
   ASSERT_OK(v->View(ty1, &w));
-  ASSERT_OK(ValidateArray(*w));
+  ASSERT_OK(w->Validate());
   AssertArraysEqual(*arr, *w);
 }
 
@@ -386,7 +386,7 @@ TEST(TestArrayView, NonZeroNestedOffset) {
   std::shared_ptr<Array> list_array;
   ASSERT_OK(ListArray::FromArrays(*list_offsets, *list_values, default_memory_pool(),
                                   &list_array));
-  ASSERT_OK(ValidateArray(*list_array));
+  ASSERT_OK(list_array->Validate());
   CheckViewFails(list_array, list(uint16()));
 }
 
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index 3c87280..01681c8 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -172,6 +172,9 @@ std::string Array::ToString() const {
   return ss.str();
 }
 
+// ----------------------------------------------------------------------
+// NullArray
+
 NullArray::NullArray(int64_t length) {
   SetData(ArrayData::Make(null(), length, {nullptr}, length));
 }
@@ -1239,7 +1242,7 @@ struct ValidateVisitor {
     if (!array.keys()) {
       return Status::Invalid("keys was null");
     }
-    const Status key_valid = ValidateArray(*array.values());
+    const Status key_valid = array.keys()->Validate();
     if (!key_valid.ok()) {
       return Status::Invalid("key array invalid: ", key_valid.ToString());
     }
@@ -1247,7 +1250,7 @@ struct ValidateVisitor {
     if (!array.values()) {
       return Status::Invalid("values was null");
     }
-    const Status values_valid = ValidateArray(*array.values());
+    const Status values_valid = array.values()->Validate();
     if (!values_valid.ok()) {
       return Status::Invalid("values array invalid: ", values_valid.ToString());
     }
@@ -1290,7 +1293,7 @@ struct ValidateVisitor {
                                  it->type()->ToString(), " at position [", idx, "]");
         }
 
-        const Status child_valid = ValidateArray(*it);
+        const Status child_valid = it->Validate();
         if (!child_valid.ok()) {
           return Status::Invalid("Child array invalid: ", child_valid.ToString(),
                                  " at position [", idx, "}");
@@ -1326,7 +1329,7 @@ struct ValidateVisitor {
                              "' has storage array of incompatible type '",
                              array.storage()->type()->ToString(), "'");
     }
-    return ValidateArray(*array.storage());
+    return array.storage()->Validate();
   }
 
  protected:
@@ -1342,7 +1345,7 @@ struct ValidateVisitor {
                              last_offset, "!=", array.values()->length());
     }
 
-    const Status child_valid = ValidateArray(*array.values());
+    const Status child_valid = array.values()->Validate();
     if (!child_valid.ok()) {
       return Status::Invalid("Child array invalid: ", child_valid.ToString());
     }
@@ -1387,17 +1390,17 @@ struct ValidateVisitor {
 
 }  // namespace internal
 
-Status ValidateArray(const Array& array) {
+Status Array::Validate() const {
   // First check the array layout conforms to the spec
-  const DataType& type = *array.type();
+  const DataType& type = *this->type();
   const auto layout = type.layout();
-  const ArrayData& data = *array.data();
+  const ArrayData& data = *this->data();
 
-  if (array.length() < 0) {
+  if (length() < 0) {
     return Status::Invalid("Array length is negative");
   }
 
-  if (array.null_count() > array.length()) {
+  if (null_count() > length()) {
     return Status::Invalid("Null count exceeds array length");
   }
 
@@ -1423,9 +1426,11 @@ Status ValidateArray(const Array& array) {
   }
 
   internal::ValidateVisitor validate_visitor;
-  return VisitArrayInline(array, &validate_visitor);
+  return VisitArrayInline(*this, &validate_visitor);
 }
 
+Status ValidateArray(const Array& array) { return array.Validate(); }
+
 // ----------------------------------------------------------------------
 // Loading from ArrayData
 
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index fbbe369..3303c82 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -343,6 +343,14 @@ class ARROW_EXPORT Array {
   /// \return PrettyPrint representation of array suitable for debugging
   std::string ToString() const;
 
+  /// \brief Perform any validation checks to determine obvious inconsistencies
+  /// within the array's internal data.
+  ///
+  /// This can be an expensive check, potentially O(length).
+  ///
+  /// \return Status
+  Status Validate() const;
+
  protected:
   Array() : null_bitmap_data_(NULLPTR) {}
 
@@ -1229,13 +1237,11 @@ class ARROW_EXPORT DictionaryArray : public Array {
   std::shared_ptr<Array> indices_;
 };
 
-/// \brief Perform any validation checks to determine obvious inconsistencies
-/// with the array's internal data
-///
-/// This can be an expensive check.
+/// \brief Alias of Array::Validate().
 ///
 /// \param array an Array instance
 /// \return Status
+ARROW_DEPRECATED("Use Array::Validate instead")
 ARROW_EXPORT
 Status ValidateArray(const Array& array);
 
diff --git a/cpp/src/arrow/array/concatenate-test.cc b/cpp/src/arrow/array/concatenate-test.cc
index 4d16b17..befe97b 100644
--- a/cpp/src/arrow/array/concatenate-test.cc
+++ b/cpp/src/arrow/array/concatenate-test.cc
@@ -148,7 +148,7 @@ TYPED_TEST(PrimitiveConcatenateTest, Primitives) {
 TEST_F(ConcatenateTest, StringType) {
   Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
     *out = rng_.String(size, /*min_length =*/0, /*max_length =*/15, null_probability);
-    ASSERT_OK(ValidateArray(**out));
+    ASSERT_OK((**out).Validate());
   });
 }
 
@@ -156,7 +156,7 @@ TEST_F(ConcatenateTest, LargeStringType) {
   Check([this](int32_t size, double null_probability, std::shared_ptr<Array>* out) {
     *out =
         rng_.LargeString(size, /*min_length =*/0, /*max_length =*/15, null_probability);
-    ASSERT_OK(ValidateArray(**out));
+    ASSERT_OK((**out).Validate());
   });
 }
 
@@ -171,7 +171,7 @@ TEST_F(ConcatenateTest, ListType) {
     std::shared_ptr<Array> offsets;
     ArrayFromVector<Int32Type>(offsets_vector, &offsets);
     ASSERT_OK(ListArray::FromArrays(*offsets, *values, default_memory_pool(), out));
-    ASSERT_OK(ValidateArray(**out));
+    ASSERT_OK((**out).Validate());
   });
 }
 
@@ -186,7 +186,7 @@ TEST_F(ConcatenateTest, LargeListType) {
     std::shared_ptr<Array> offsets;
     ArrayFromVector<Int64Type>(offsets_vector, &offsets);
     ASSERT_OK(LargeListArray::FromArrays(*offsets, *values, default_memory_pool(), out));
-    ASSERT_OK(ValidateArray(**out));
+    ASSERT_OK((**out).Validate());
   });
 }
 
diff --git a/cpp/src/arrow/compute/kernels/argsort-test.cc b/cpp/src/arrow/compute/kernels/argsort-test.cc
index d879139..5a54cd4 100644
--- a/cpp/src/arrow/compute/kernels/argsort-test.cc
+++ b/cpp/src/arrow/compute/kernels/argsort-test.cc
@@ -37,7 +37,7 @@ class TestArgsortKernel : public ComputeFixture, public TestBase {
                            const std::shared_ptr<Array> expected) {
     std::shared_ptr<Array> actual;
     ASSERT_OK(arrow::compute::Argsort(&this->ctx_, *values, &actual));
-    ASSERT_OK(ValidateArray(*actual));
+    ASSERT_OK(actual->Validate());
     AssertArraysEqual(*expected, *actual);
   }
 
diff --git a/cpp/src/arrow/compute/kernels/cast-test.cc b/cpp/src/arrow/compute/kernels/cast-test.cc
index 523b0b8..71b35b7 100644
--- a/cpp/src/arrow/compute/kernels/cast-test.cc
+++ b/cpp/src/arrow/compute/kernels/cast-test.cc
@@ -69,7 +69,7 @@ class TestCast : public ComputeFixture, public TestBase {
                  const std::shared_ptr<DataType>& out_type, const CastOptions& options) {
     std::shared_ptr<Array> result;
     ASSERT_OK(Cast(&ctx_, input, out_type, options, &result));
-    ASSERT_OK(ValidateArray(*result));
+    ASSERT_OK(result->Validate());
     ASSERT_ARRAYS_EQUAL(expected, *result);
   }
 
@@ -89,7 +89,7 @@ class TestCast : public ComputeFixture, public TestBase {
   void CheckZeroCopy(const Array& input, const std::shared_ptr<DataType>& out_type) {
     std::shared_ptr<Array> result;
     ASSERT_OK(Cast(&ctx_, input, out_type, {}, &result));
-    ASSERT_OK(ValidateArray(*result));
+    ASSERT_OK(result->Validate());
     ASSERT_EQ(input.data()->buffers.size(), result->data()->buffers.size());
     for (size_t i = 0; i < input.data()->buffers.size(); ++i) {
       AssertBufferSame(input, *result, static_cast<int>(i));
@@ -1260,7 +1260,7 @@ TYPED_TEST(TestNullCast, FromNull) {
 
   std::shared_ptr<Array> result;
   ASSERT_OK(Cast(&this->ctx_, arr, out_type, {}, &result));
-  ASSERT_OK(ValidateArray(*result));
+  ASSERT_OK(result->Validate());
 
   ASSERT_TRUE(result->type()->Equals(*out_type));
   ASSERT_EQ(length, result->length());
@@ -1314,7 +1314,7 @@ TYPED_TEST(TestDictionaryCast, NoNulls) {
   data->buffers[0] = nullptr;
   data->null_count = 0;
   std::shared_ptr<Array> dict_array = std::make_shared<DictionaryArray>(data);
-  ASSERT_OK(ValidateArray(*dict_array));
+  ASSERT_OK(dict_array->Validate());
 
   this->CheckPass(*dict_array, *plain_array, plain_array->type(), options);
 }
diff --git a/cpp/src/arrow/compute/kernels/filter-test.cc b/cpp/src/arrow/compute/kernels/filter-test.cc
index fa40753..ec35aa3 100644
--- a/cpp/src/arrow/compute/kernels/filter-test.cc
+++ b/cpp/src/arrow/compute/kernels/filter-test.cc
@@ -44,7 +44,7 @@ class TestFilterKernel : public ComputeFixture, public TestBase {
                           const std::shared_ptr<Array>& expected) {
     std::shared_ptr<Array> actual;
     ASSERT_OK(arrow::compute::Filter(&this->ctx_, *values, *filter, &actual));
-    ASSERT_OK(ValidateArray(*actual));
+    ASSERT_OK(actual->Validate());
     AssertArraysEqual(*expected, *actual);
   }
 
@@ -52,7 +52,7 @@ class TestFilterKernel : public ComputeFixture, public TestBase {
                     const std::string& filter, const std::string& expected) {
     std::shared_ptr<Array> actual;
     ASSERT_OK(this->Filter(type, values, filter, &actual));
-    ASSERT_OK(ValidateArray(*actual));
+    ASSERT_OK(actual->Validate());
     AssertArraysEqual(*ArrayFromJSON(type, expected), *actual);
   }
 
@@ -66,7 +66,7 @@ class TestFilterKernel : public ComputeFixture, public TestBase {
                       const std::shared_ptr<Array>& filter_boxed) {
     std::shared_ptr<Array> filtered;
     ASSERT_OK(arrow::compute::Filter(&this->ctx_, *values, *filter_boxed, &filtered));
-    ASSERT_OK(ValidateArray(*filtered));
+    ASSERT_OK(filtered->Validate());
 
     auto filter = checked_pointer_cast<BooleanArray>(filter_boxed);
     int64_t values_i = 0, filtered_i = 0;
@@ -222,7 +222,7 @@ TYPED_TEST(TestFilterKernelWithNumeric, CompareScalarAndFilterRandomNumeric) {
                                         &selection));
       ASSERT_OK(arrow::compute::Filter(&this->ctx_, Datum(array), selection, &filtered));
       auto filtered_array = filtered.make_array();
-      ASSERT_OK(ValidateArray(*filtered_array));
+      ASSERT_OK(filtered_array->Validate());
       auto expected =
           CompareAndFilter<TypeParam>(array->raw_values(), array->length(), c_fifty, op);
       ASSERT_ARRAYS_EQUAL(*filtered_array, *expected);
@@ -247,7 +247,7 @@ TYPED_TEST(TestFilterKernelWithNumeric, CompareArrayAndFilterRandomNumeric) {
                                         &selection));
       ASSERT_OK(arrow::compute::Filter(&this->ctx_, Datum(lhs), selection, &filtered));
       auto filtered_array = filtered.make_array();
-      ASSERT_OK(ValidateArray(*filtered_array));
+      ASSERT_OK(filtered_array->Validate());
       auto expected = CompareAndFilter<TypeParam>(lhs->raw_values(), lhs->length(),
                                                   rhs->raw_values(), op);
       ASSERT_ARRAYS_EQUAL(*filtered_array, *expected);
@@ -277,7 +277,7 @@ TYPED_TEST(TestFilterKernelWithNumeric, ScalarInRangeAndFilterRandomNumeric) {
                                   &selection));
     ASSERT_OK(arrow::compute::Filter(&this->ctx_, Datum(array), selection, &filtered));
     auto filtered_array = filtered.make_array();
-    ASSERT_OK(ValidateArray(*filtered_array));
+    ASSERT_OK(filtered_array->Validate());
     auto expected = CompareAndFilter<TypeParam>(
         array->raw_values(), array->length(),
         [&](CType e) { return (e > c_fifty) && (e < c_hundred); });
diff --git a/cpp/src/arrow/compute/kernels/take-test.cc b/cpp/src/arrow/compute/kernels/take-test.cc
index 0f080fd..a4db095 100644
--- a/cpp/src/arrow/compute/kernels/take-test.cc
+++ b/cpp/src/arrow/compute/kernels/take-test.cc
@@ -44,7 +44,7 @@ class TestTakeKernel : public ComputeFixture, public TestBase {
     std::shared_ptr<Array> actual;
     TakeOptions options;
     ASSERT_OK(arrow::compute::Take(&this->ctx_, *values, *indices, options, &actual));
-    ASSERT_OK(ValidateArray(*actual));
+    ASSERT_OK(actual->Validate());
     AssertArraysEqual(*expected, *actual);
   }
 
@@ -54,7 +54,7 @@ class TestTakeKernel : public ComputeFixture, public TestBase {
 
     for (auto index_type : {int8(), uint32()}) {
       ASSERT_OK(this->Take(type, values, index_type, indices, &actual));
-      ASSERT_OK(ValidateArray(*actual));
+      ASSERT_OK(actual->Validate());
       AssertArraysEqual(*ArrayFromJSON(type, expected), *actual);
     }
   }
@@ -131,7 +131,7 @@ class TestTakeKernelWithNumeric : public TestTakeKernel<ArrowType> {
     TakeOptions options;
     ASSERT_OK(
         arrow::compute::Take(&this->ctx_, *values, *indices_boxed, options, &taken));
-    ASSERT_OK(ValidateArray(*taken));
+    ASSERT_OK(taken->Validate());
     ASSERT_EQ(indices_boxed->length(), taken->length());
 
     ASSERT_EQ(indices_boxed->type_id(), Type::INT32);
@@ -415,7 +415,7 @@ class TestPermutationsWithTake : public ComputeFixture, public TestBase {
     TakeOptions options;
     std::shared_ptr<Array> boxed_out;
     ASSERT_OK(arrow::compute::Take(&this->ctx_, values, indices, options, &boxed_out));
-    ASSERT_OK(ValidateArray(*boxed_out));
+    ASSERT_OK(boxed_out->Validate());
     *out = checked_pointer_cast<Int16Array>(std::move(boxed_out));
   }
 
diff --git a/cpp/src/arrow/flight/perf-server.cc b/cpp/src/arrow/flight/perf-server.cc
index 43cede4..60b2d4e 100644
--- a/cpp/src/arrow/flight/perf-server.cc
+++ b/cpp/src/arrow/flight/perf-server.cc
@@ -130,7 +130,7 @@ Status GetPerfBatches(const perf::Token& token, const std::shared_ptr<Schema>& s
     RETURN_NOT_OK(MakeRandomByteBuffer(length * sizeof(int64_t), default_memory_pool(),
                                        &buffer, static_cast<int32_t>(i) /* seed */));
     arrays.push_back(std::make_shared<Int64Array>(length, buffer));
-    RETURN_NOT_OK(ValidateArray(*arrays.back()));
+    RETURN_NOT_OK(arrays.back()->Validate());
   }
 
   *data_stream = std::unique_ptr<FlightDataStream>(
diff --git a/cpp/src/arrow/ipc/json-simple-test.cc b/cpp/src/arrow/ipc/json-simple-test.cc
index c202402..2628373 100644
--- a/cpp/src/arrow/ipc/json-simple-test.cc
+++ b/cpp/src/arrow/ipc/json-simple-test.cc
@@ -93,7 +93,7 @@ void AssertJSONArray(const std::shared_ptr<DataType>& type, const std::string& j
   std::shared_ptr<Array> actual, expected;
 
   ASSERT_OK(ArrayFromJSON(type, json, &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<T, C_TYPE>(type, values, &expected);
   AssertArraysEqual(*expected, *actual);
 }
@@ -105,7 +105,7 @@ void AssertJSONArray(const std::shared_ptr<DataType>& type, const std::string& j
   std::shared_ptr<Array> actual, expected;
 
   ASSERT_OK(ArrayFromJSON(type, json, &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<T, C_TYPE>(type, is_valid, values, &expected);
   AssertArraysEqual(*expected, *actual);
 }
@@ -259,7 +259,7 @@ TEST(TestFloat, Basics) {
   // Check NaN separately as AssertArraysEqual simply memcmp's array contents
   // and NaNs can have many bit representations.
   ASSERT_OK(ArrayFromJSON(type, "[NaN]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   float value = checked_cast<FloatArray&>(*actual).Value(0);
   ASSERT_TRUE(std::isnan(value));
 }
@@ -281,7 +281,7 @@ TEST(TestDouble, Basics) {
                               {-0.0, INFINITY, -INFINITY, 0.0});
 
   ASSERT_OK(ArrayFromJSON(type, "[NaN]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   double value = checked_cast<DoubleArray&>(*actual).Value(0);
   ASSERT_TRUE(std::isnan(value));
 }
@@ -388,7 +388,7 @@ TEST(TestDecimal, Basics) {
   std::shared_ptr<Array> expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     Decimal128Builder builder(type);
     ASSERT_OK(builder.Finish(&expected));
@@ -396,7 +396,7 @@ TEST(TestDecimal, Basics) {
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[\"123.4567\", \"-78.9000\"]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     Decimal128Builder builder(type);
     ASSERT_OK(builder.Append(Decimal128(1234567)));
@@ -406,7 +406,7 @@ TEST(TestDecimal, Basics) {
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[\"123.4567\", null]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     Decimal128Builder builder(type);
     ASSERT_OK(builder.Append(Decimal128(1234567)));
@@ -433,21 +433,21 @@ TEST(TestList, IntegerList) {
   std::shared_ptr<Array> offsets, values, expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int32Type>({0}, &offsets);
   ArrayFromVector<Int64Type>({}, &values);
   ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[4, 5], [], [6]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int32Type>({0, 2, 2, 3}, &offsets);
   ArrayFromVector<Int64Type>({4, 5, 6}, &values);
   ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[], [null], [6, null]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int32Type>({0, 0, 1, 3}, &offsets);
   auto is_valid = std::vector<bool>{false, true, false};
   ArrayFromVector<Int64Type>(is_valid, {0, 6, 0}, &values);
@@ -455,7 +455,7 @@ TEST(TestList, IntegerList) {
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [], null]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     std::unique_ptr<ArrayBuilder> builder;
     ASSERT_OK(MakeBuilder(pool, type, &builder));
@@ -483,21 +483,21 @@ TEST(TestList, NullList) {
   std::shared_ptr<Array> offsets, values, expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int32Type>({0}, &offsets);
   values = std::make_shared<NullArray>(0);
   ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[], [null], [null, null]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int32Type>({0, 0, 1, 3}, &offsets);
   values = std::make_shared<NullArray>(3);
   ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &expected));
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [], null]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     std::unique_ptr<ArrayBuilder> builder;
     ASSERT_OK(MakeBuilder(pool, type, &builder));
@@ -516,7 +516,7 @@ TEST(TestList, IntegerListList) {
   std::shared_ptr<Array> offsets, values, nested, expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[[[4], [5, 6]], [[7, 8, 9]]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int32Type>({0, 1, 3, 6}, &offsets);
   ArrayFromVector<UInt8Type>({4, 5, 6, 7, 8, 9}, &values);
   ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &nested));
@@ -526,7 +526,7 @@ TEST(TestList, IntegerListList) {
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[], [[]], [[4], [], [5, 6]], [[7, 8, 9]]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int32Type>({0, 0, 1, 1, 3, 6}, &offsets);
   ArrayFromVector<UInt8Type>({4, 5, 6, 7, 8, 9}, &values);
   ASSERT_OK(ListArray::FromArrays(*offsets, *values, pool, &nested));
@@ -536,7 +536,7 @@ TEST(TestList, IntegerListList) {
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [null], [[null]]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     std::unique_ptr<ArrayBuilder> builder;
     ASSERT_OK(MakeBuilder(pool, type, &builder));
@@ -558,7 +558,7 @@ TEST(TestLargeList, Basics) {
   std::shared_ptr<Array> offsets, values, expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[[], [null], [6, null]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int64Type>({0, 0, 1, 3}, &offsets);
   auto is_valid = std::vector<bool>{false, true, false};
   ArrayFromVector<Int16Type>(is_valid, {0, 6, 0}, &values);
@@ -707,26 +707,26 @@ TEST(TestFixedSizeList, IntegerList) {
   std::shared_ptr<Array> values, expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int64Type>({}, &values);
   expected = std::make_shared<FixedSizeListArray>(type, 0, values);
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[4, 5], [0, 0], [6, 7]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int64Type>({4, 5, 0, 0, 6, 7}, &values);
   expected = std::make_shared<FixedSizeListArray>(type, 3, values);
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[null, null], [0, null], [6, null]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   auto is_valid = std::vector<bool>{false, false, true, false, true, false};
   ArrayFromVector<Int64Type>(is_valid, {0, 0, 0, 0, 6, 0}, &values);
   expected = std::make_shared<FixedSizeListArray>(type, 3, values);
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [null, null], null]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     std::unique_ptr<ArrayBuilder> builder;
     ASSERT_OK(MakeBuilder(pool, type, &builder));
@@ -758,19 +758,19 @@ TEST(TestFixedSizeList, NullList) {
   std::shared_ptr<Array> values, expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   values = std::make_shared<NullArray>(0);
   expected = std::make_shared<FixedSizeListArray>(type, 0, values);
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[null, null], [null, null], [null, null]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   values = std::make_shared<NullArray>(6);
   expected = std::make_shared<FixedSizeListArray>(type, 3, values);
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [null, null], null]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     std::unique_ptr<ArrayBuilder> builder;
     ASSERT_OK(MakeBuilder(pool, type, &builder));
@@ -793,14 +793,14 @@ TEST(TestFixedSizeList, IntegerListList) {
   std::shared_ptr<Array> values, nested, expected, actual;
 
   ASSERT_OK(ArrayFromJSON(type, "[[[1, 4]], [[2, 5]], [[3, 6]]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<UInt8Type>({1, 4, 2, 5, 3, 6}, &values);
   nested = std::make_shared<FixedSizeListArray>(nested_type, 3, values);
   expected = std::make_shared<FixedSizeListArray>(type, 3, nested);
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[[1, null]], [null], null]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   {
     std::unique_ptr<ArrayBuilder> builder;
     ASSERT_OK(MakeBuilder(pool, type, &builder));
@@ -835,7 +835,7 @@ TEST(TestStruct, SimpleStruct) {
 
   // Trivial
   ASSERT_OK(ArrayFromJSON(type, "[]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int8Type>({}, &a);
   ArrayFromVector<BooleanType, bool>({}, &b);
   children.assign({a, b});
@@ -849,11 +849,11 @@ TEST(TestStruct, SimpleStruct) {
   expected = std::make_shared<StructArray>(type, 2, children);
 
   ASSERT_OK(ArrayFromJSON(type, "[[5, true], [6, false]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   AssertArraysEqual(*expected, *actual);
   ASSERT_OK(ArrayFromJSON(type, "[{\"a\": 5, \"b\": true}, {\"b\": false, \"a\": 6}]",
                           &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   AssertArraysEqual(*expected, *actual);
 
   // With nulls
@@ -867,12 +867,12 @@ TEST(TestStruct, SimpleStruct) {
 
   ASSERT_OK(
       ArrayFromJSON(type, "[null, [5, null], [null, false], [null, null]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   AssertArraysEqual(*expected, *actual);
   // When using object notation, null members can be omitted
   ASSERT_OK(ArrayFromJSON(type, "[null, {\"a\": 5, \"b\": null}, {\"b\": false}, {}]",
                           &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   AssertArraysEqual(*expected, *actual);
 }
 
@@ -889,7 +889,7 @@ TEST(TestStruct, NestedStruct) {
   std::vector<std::shared_ptr<Array>> children(2);
 
   ASSERT_OK(ArrayFromJSON(type, "[]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int8Type>({}, &children[0]);
   ArrayFromVector<BooleanType, bool>({}, &children[1]);
   children[0] = std::make_shared<StructArray>(nested_type, 0, children);
@@ -898,7 +898,7 @@ TEST(TestStruct, NestedStruct) {
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[[[5, true], 1.5], [[6, false], -3e2]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   ArrayFromVector<Int8Type>({5, 6}, &children[0]);
   ArrayFromVector<BooleanType, bool>({true, false}, &children[1]);
   children[0] = std::make_shared<StructArray>(nested_type, 2, children);
@@ -907,7 +907,7 @@ TEST(TestStruct, NestedStruct) {
   AssertArraysEqual(*expected, *actual);
 
   ASSERT_OK(ArrayFromJSON(type, "[null, [[5, null], null], [null, -3e2]]", &actual));
-  ASSERT_OK(ValidateArray(*actual));
+  ASSERT_OK(actual->Validate());
   is_valid = {false, true, false};
   ArrayFromVector<Int8Type>(is_valid, {0, 5, 0}, &children[0]);
   is_valid = {false, false, false};
diff --git a/cpp/src/arrow/ipc/test-common.cc b/cpp/src/arrow/ipc/test-common.cc
index 1cb40d9..cc624d1 100644
--- a/cpp/src/arrow/ipc/test-common.cc
+++ b/cpp/src/arrow/ipc/test-common.cc
@@ -118,7 +118,7 @@ static Status MakeListArray(const std::shared_ptr<Array>& child_array, int num_l
   *out = std::make_shared<ArrayType>(std::make_shared<TypeClass>(child_array->type()),
                                      num_lists, offsets_buffer, child_array, null_bitmap,
                                      kUnknownNullCount);
-  return ValidateArray(**out);
+  return (**out).Validate();
 }
 
 Status MakeRandomListArray(const std::shared_ptr<Array>& child_array, int num_lists,
diff --git a/cpp/src/parquet/arrow/arrow-reader-writer-test.cc b/cpp/src/parquet/arrow/arrow-reader-writer-test.cc
index ee4ddcb..1d50bcb 100644
--- a/cpp/src/parquet/arrow/arrow-reader-writer-test.cc
+++ b/cpp/src/parquet/arrow/arrow-reader-writer-test.cc
@@ -2270,6 +2270,7 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
       }
     }
     ASSERT_EQ(local_null_count, expected_nulls);
+    ASSERT_OK(array.Validate());
   }
 
   void ValidateColumnArray(const ::arrow::Int32Array& array, size_t expected_nulls) {
diff --git a/python/pyarrow/array.pxi b/python/pyarrow/array.pxi
index cd6c190..6e24dbc 100644
--- a/python/pyarrow/array.pxi
+++ b/python/pyarrow/array.pxi
@@ -838,15 +838,15 @@ cdef class Array(_PandasConvertible):
     def validate(self):
         """
         Perform any validation checks implemented by
-        arrow::ValidateArray. Raises exception with error message if array does
-        not validate
+        arrow::Array::Validate(). Raises exception with error message if
+        array does not validate.
 
         Raises
         ------
         ArrowInvalid
         """
         with nogil:
-            check_status(ValidateArray(deref(self.ap)))
+            check_status(self.ap.Validate())
 
     @property
     def offset(self):
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 9c2aa6a..d3eca97 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -151,6 +151,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         shared_ptr[CArray] Slice(int64_t offset)
         shared_ptr[CArray] Slice(int64_t offset, int64_t length)
 
+        CStatus Validate() const
+
     shared_ptr[CArray] MakeArray(const shared_ptr[CArrayData]& data)
 
     CStatus DebugPrint(const CArray& arr, int indent)
@@ -518,8 +520,6 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
         CStatus Flatten(CMemoryPool* pool, vector[shared_ptr[CArray]]* out)
 
-    CStatus ValidateArray(const CArray& array)
-
     cdef cppclass CChunkedArray" arrow::ChunkedArray":
         CChunkedArray(const vector[shared_ptr[CArray]]& arrays)
         CChunkedArray(const vector[shared_ptr[CArray]]& arrays,