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

[GitHub] [arrow] pitrou commented on a diff in pull request #35197: GH-14946: [C++] Add flattening FieldPath/FieldRef::Get methods

pitrou commented on code in PR #35197:
URL: https://github.com/apache/arrow/pull/35197#discussion_r1194041108


##########
cpp/src/arrow/type.h:
##########
@@ -1698,10 +1698,45 @@ class ARROW_EXPORT FieldPath {
   /// \brief Retrieve the referenced child from a ChunkedArray
   Result<std::shared_ptr<ChunkedArray>> Get(const ChunkedArray& chunked_array) const;
 
+  /// \brief Retrieve the referenced child/column from an Array, ArrayData, ChunkedArray,
+  /// RecordBatch, or Table
+  ///
+  /// Unlike `FieldPath::Get`, these variants are not zero-copy and the retrieved child's
+  /// null bitmap is ANDed with its parent's
+  Result<std::shared_ptr<Array>> GetFlattened(const Array& array,
+                                              MemoryPool* pool = NULLPTR) const;
+  Result<std::shared_ptr<ArrayData>> GetFlattened(const ArrayData& data,
+                                                  MemoryPool* pool = NULLPTR) const;
+  Result<std::shared_ptr<ChunkedArray>> GetFlattened(const ChunkedArray& chunked_array,
+                                                     MemoryPool* pool = NULLPTR) const;
+  Result<std::shared_ptr<Array>> GetFlattened(const RecordBatch& batch,
+                                              MemoryPool* pool = NULLPTR) const;
+  Result<std::shared_ptr<ChunkedArray>> GetFlattened(const Table& table,
+                                                     MemoryPool* pool = NULLPTR) const;
+
  private:
   std::vector<int> indices_;
 };
 
+namespace internal {
+
+template <typename T>
+using FieldPathGetType =
+    decltype(std::declval<FieldPath>().Get(std::declval<T>()).ValueOrDie());
+
+template <bool Flattened, typename T>
+std::enable_if_t<!Flattened, Result<FieldPathGetType<T>>> GetChild(
+    const T& root, const FieldPath& path, MemoryPool* = NULLPTR) {
+  return path.Get(root);
+}
+template <bool Flattened, typename T>
+std::enable_if_t<Flattened, Result<FieldPathGetType<T>>> GetChild(
+    const T& root, const FieldPath& path, MemoryPool* pool = NULLPTR) {
+  return path.GetFlattened(root, pool);
+}

Review Comment:
   I'm not sure the indirection through these helpers and the `Flattened` templatization is useful (instead of simply duplicating the definition of e.g. `FieldRef::GetOne` into `FieldRef::GetOneFlattened`).
   
   But if it is, we can probably shorten this helper code with C++17 features:
   1) using `auto` and return type deduction to avoid the `decltype` dance
   2) using `if constexpr` to avoid SFINAE (not 100% it will work here, but worth a try)
   



##########
cpp/src/arrow/type.h:
##########
@@ -1698,10 +1698,45 @@ class ARROW_EXPORT FieldPath {
   /// \brief Retrieve the referenced child from a ChunkedArray
   Result<std::shared_ptr<ChunkedArray>> Get(const ChunkedArray& chunked_array) const;
 
+  /// \brief Retrieve the referenced child/column from an Array, ArrayData, ChunkedArray,
+  /// RecordBatch, or Table
+  ///
+  /// Unlike `FieldPath::Get`, these variants are not zero-copy and the retrieved child's
+  /// null bitmap is ANDed with its parent's

Review Comment:
   Perhaps "ancestors" plural?
   ```suggestion
     /// null bitmap is ANDed with its ancestors'
   ```



##########
cpp/src/arrow/type_test.cc:
##########
@@ -364,152 +365,344 @@ TEST(TestField, TestMerge) {
   }
 }
 
-TEST(TestFieldPath, Basics) {
-  auto f0 = field("alpha", int32());
-  auto f1 = field("beta", int32());
-  auto f2 = field("alpha", int32());
-  auto f3 = field("beta", int32());
-  Schema s({f0, f1, f2, f3});
+struct FieldPathTestCase {
+  struct OutputValues {
+    explicit OutputValues(std::vector<int> indices = {})
+        : path(FieldPath(std::move(indices))) {}
 
-  // retrieving a field with single-element FieldPath is equivalent to Schema::field
-  for (int index = 0; index < s.num_fields(); ++index) {
-    ASSERT_OK_AND_EQ(s.field(index), FieldPath({index}).Get(s));
+    template <typename T>
+    auto&& Get() const;
+
+    FieldPath path;
+    std::shared_ptr<Field> field;
+    std::shared_ptr<Array> array;
+    std::shared_ptr<ChunkedArray> chunked_array;
+  };
+
+  static constexpr int kNumColumns = 2;
+  static constexpr int kNumRows = 100;
+  static constexpr int kRandomSeed = 0xbeef;
+
+  // Input for the FieldPath::Get functions in multiple forms
+  std::shared_ptr<Schema> schema;
+  std::shared_ptr<DataType> type;
+  std::shared_ptr<Array> array;
+  std::shared_ptr<RecordBatch> record_batch;
+  std::shared_ptr<ChunkedArray> chunked_array;
+  std::shared_ptr<Table> table;
+
+  template <typename T>
+  auto&& GetInput() const;
+
+  // Number of chunks for each column in the input Table
+  const std::array<int, kNumColumns> num_column_chunks = {15, 20};
+  // Number of chunks in the input ChunkedArray
+  const int num_chunks = 15;
+
+  // Expected outputs for each child;
+  OutputValues v0{{0}}, v1{{1}};
+  OutputValues v1_0{{1, 0}}, v1_1{{1, 1}};
+  OutputValues v1_1_0{{1, 1, 0}}, v1_1_1{{1, 1, 1}};
+  // Expected outputs for nested children with null flattening applied
+  OutputValues v1_0_flat{{1, 0}}, v1_1_flat{{1, 1}};
+  OutputValues v1_1_0_flat{{1, 1, 0}}, v1_1_1_flat{{1, 1, 1}};
+
+  static const FieldPathTestCase* Instance() {
+    static const auto maybe_instance = Make();
+    return &maybe_instance.ValueOrDie();
   }
-  EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid,
-                                  testing::HasSubstr("empty indices cannot be traversed"),
-                                  FieldPath().Get(s));
-  EXPECT_RAISES_WITH_MESSAGE_THAT(IndexError, testing::HasSubstr("index out of range"),
-                                  FieldPath({s.num_fields() * 2}).Get(s));
-}
-
-TEST(TestFieldPath, GetForTable) {
-  using testing::HasSubstr;
-
-  constexpr int kNumRows = 4;
-  auto f0 = field("a", int32());
-  auto f1 = field("b", int32());
-  auto f2 = field("c", struct_({f1}));
-  auto f3 = field("d", struct_({f0, f2}));
-  auto table_schema = schema({f0, f1, f2, f3});
-
-  // Each column has a different chunking
-  ChunkedArrayVector columns(4);
-  columns[0] = ChunkedArrayFromJSON(f0->type(), {"[0,1,2,3]"});
-  columns[1] = ChunkedArrayFromJSON(f1->type(), {"[3,2,1]", "[0]"});
-  columns[2] =
-      ChunkedArrayFromJSON(f2->type(), {R"([{"b":3},{"b":2}])", R"([{"b":1},{"b":0}])"});
-  columns[3] = ChunkedArrayFromJSON(
-      f3->type(), {R"([{"a":0,"c":{"b":3}},{"a":1,"c":{"b":2}}])",
-                   R"([{"a":2,"c":{"b":1}}])", R"([{"a":3,"c":{"b":0}}])"});
-  auto table = Table::Make(table_schema, columns, kNumRows);
-  ASSERT_OK(table->ValidateFull());
 
-  ASSERT_OK_AND_ASSIGN(auto v0, FieldPath({0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v1, FieldPath({1}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v2, FieldPath({2}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v2_0, FieldPath({2, 0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3, FieldPath({3}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_0, FieldPath({3, 0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_1, FieldPath({3, 1}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_1_0, FieldPath({3, 1, 0}).Get(*table));
-
-  EXPECT_EQ(v0->num_chunks(), columns[0]->num_chunks());
-  EXPECT_EQ(v1->num_chunks(), columns[1]->num_chunks());
-  EXPECT_EQ(v2->num_chunks(), columns[2]->num_chunks());
-  EXPECT_EQ(v2_0->num_chunks(), columns[2]->num_chunks());
-  EXPECT_EQ(v3->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_0->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_1->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_1_0->num_chunks(), columns[3]->num_chunks());
-
-  EXPECT_TRUE(columns[0]->Equals(v0));
-  EXPECT_TRUE(columns[0]->Equals(v3_0));
-
-  EXPECT_TRUE(columns[1]->Equals(v1));
-  EXPECT_TRUE(columns[1]->Equals(v2_0));
-  EXPECT_TRUE(columns[1]->Equals(v3_1_0));
-
-  EXPECT_TRUE(columns[2]->Equals(v2));
-  EXPECT_TRUE(columns[2]->Equals(v3_1));
-
-  EXPECT_TRUE(columns[3]->Equals(v3));
-
-  for (const auto& path :
-       {FieldPath({4, 1, 0}), FieldPath({3, 2, 0}), FieldPath{3, 1, 1}}) {
-    EXPECT_RAISES_WITH_MESSAGE_THAT(IndexError, HasSubstr("index out of range"),
-                                    path.Get(*table));
+  static Result<FieldPathTestCase> Make() {
+    // Generate test input based on a single schema. First by creating a StructArray,
+    // then deriving the other input types (ChunkedArray, RecordBatch, Table, etc) from
+    // it. We also compute the expected outputs for each child individually (for each
+    // output type).
+    FieldPathTestCase out;
+    random::RandomArrayGenerator gen(kRandomSeed);
+
+    // Define child fields and input schema
+    out.v1_1_1.field = field("b", boolean());
+    out.v1_1_0.field = field("f", float32());
+    out.v1_1.field = field("s1", struct_({out.v1_1_0.field, out.v1_1_1.field}));
+    out.v1_0.field = field("i", int32());
+    out.v1.field = field("s0", struct_({out.v1_0.field, out.v1_1.field}));
+    out.v0.field = field("u", utf8());
+    out.schema = arrow::schema({out.v0.field, out.v1.field});
+    out.type = struct_(out.schema->fields());
+
+    // Create null bitmaps for the struct fields independent of its childrens'
+    // bitmaps. For FieldPath::GetFlattened, parent/child bitmaps should be combined
+    // - for FieldPath::Get, higher-level nulls are ignored.
+    auto bitmap1_1 = gen.NullBitmap(kNumRows, 0.15);
+    auto bitmap1 = gen.NullBitmap(kNumRows, 0.30);
+
+    // Generate raw leaf arrays
+    out.v1_1_1.array = gen.ArrayOf(out.v1_1_1.field->type(), kNumRows);
+    out.v1_1_0.array = gen.ArrayOf(out.v1_1_0.field->type(), kNumRows);
+    out.v1_0.array = gen.ArrayOf(out.v1_0.field->type(), kNumRows);
+    out.v0.array = gen.ArrayOf(out.v0.field->type(), kNumRows);
+    // Make struct fields from leaf arrays (we use the custom bitmaps here)
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1.array,
+        StructArray::Make({out.v1_1_0.array, out.v1_1_1.array},
+                          {out.v1_1_0.field, out.v1_1_1.field}, bitmap1_1));
+    ARROW_ASSIGN_OR_RAISE(out.v1.array,
+                          StructArray::Make({out.v1_0.array, out.v1_1.array},
+                                            {out.v1_0.field, out.v1_1.field}, bitmap1));
+
+    // Not used to create the test input, but pre-compute flattened versions of nested
+    // arrays for comparisons in the GetFlattened tests.
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_0_flat.array,
+        checked_pointer_cast<StructArray>(out.v1.array)->GetFlattenedField(0));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_flat.array,
+        checked_pointer_cast<StructArray>(out.v1.array)->GetFlattenedField(1));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_0_flat.array,
+        checked_pointer_cast<StructArray>(out.v1_1_flat.array)->GetFlattenedField(0));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_1_flat.array,
+        checked_pointer_cast<StructArray>(out.v1_1_flat.array)->GetFlattenedField(1));
+    // Sanity check
+    ARROW_CHECK(!out.v1_0_flat.array->Equals(out.v1_0.array));
+    ARROW_CHECK(!out.v1_1_flat.array->Equals(out.v1_1.array));
+    ARROW_CHECK(!out.v1_1_0_flat.array->Equals(out.v1_1_0.array));
+    ARROW_CHECK(!out.v1_1_1_flat.array->Equals(out.v1_1_1.array));
+
+    // Finalize the input Array
+    ARROW_ASSIGN_OR_RAISE(out.array, StructArray::Make({out.v0.array, out.v1.array},
+                                                       {out.v0.field, out.v1.field}));
+    ARROW_RETURN_NOT_OK(out.array->ValidateFull());
+    // Finalize the input RecordBatch
+    ARROW_ASSIGN_OR_RAISE(out.record_batch, RecordBatch::FromStructArray(out.array));
+    ARROW_RETURN_NOT_OK(out.record_batch->ValidateFull());
+    // Finalize the input ChunkedArray
+    out.chunked_array = SliceToChunkedArray(*out.array, out.num_chunks);
+    ARROW_RETURN_NOT_OK(out.chunked_array->ValidateFull());
+
+    // For each expected child array, create a chunked equivalent (we use a different
+    // chunk layout for each top-level column to make the Table test more interesting)
+    for (OutputValues* v :
+         {&out.v0, &out.v1, &out.v1_0, &out.v1_1, &out.v1_1_0, &out.v1_1_1,
+          &out.v1_0_flat, &out.v1_1_flat, &out.v1_1_0_flat, &out.v1_1_1_flat}) {
+      v->chunked_array =
+          SliceToChunkedArray(*v->array, out.num_column_chunks[v->path[0]]);
+    }
+    // Finalize the input Table
+    out.table =
+        Table::Make(out.schema, {out.v0.chunked_array, out.v1.chunked_array}, kNumRows);
+    ARROW_RETURN_NOT_OK(out.table->ValidateFull());
+
+    return std::move(out);
   }
-  EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("empty indices cannot be traversed"),
-                                  FieldPath().Get(*table));
-}
-
-TEST(TestFieldPath, GetForChunkedArray) {
-  using testing::HasSubstr;
-
-  auto f0 = field("a", int32());
-  auto f1 = field("b", int32());
-  auto f2 = field("c", struct_({f1}));
-  auto f3 = field("d", struct_({f0, f2}));
-  auto type = struct_({f0, f1, f3});
-
-  auto column0 = ChunkedArrayFromJSON(f0->type(), {"[0,1,2,3]"});
-  auto column1 = ChunkedArrayFromJSON(f1->type(), {"[3,2,1,0]"});
-  auto column2_1 =
-      ChunkedArrayFromJSON(f2->type(), {R"([{"b":3},{"b":2},{"b":1},{"b":0}])"});
-  auto chunked_array = ChunkedArrayFromJSON(
-      type,
-      {
-          R"([{"a":0,"b":3,"d":{"a":0,"c":{"b":3}}}])",
-          R"([{"a":1,"b":2,"d":{"a":1,"c":{"b":2}}},{"a":2,"b":1,"d":{"a":2,"c":{"b":1}}}])",
-          R"([{"a":3,"b":0,"d":{"a":3,"c":{"b":0}}}])",
-      });
-  ASSERT_OK(chunked_array->ValidateFull());
-
-  ASSERT_OK_AND_ASSIGN(auto v0, FieldPath({0}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v1, FieldPath({1}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_0, FieldPath({2, 0}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_1, FieldPath({2, 1}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_1_0, FieldPath({2, 1, 0}).Get(*chunked_array));
-
-  for (const auto& v : {v0, v1, v2_0, v2_1, v2_1_0}) {
-    EXPECT_EQ(v->num_chunks(), chunked_array->num_chunks());
+
+  static std::shared_ptr<ChunkedArray> SliceToChunkedArray(const Array& array,
+                                                           int num_chunks) {
+    ARROW_CHECK(num_chunks > 0 && array.length() >= num_chunks);

Review Comment:
   Generally we should favor `ASSERT` or `EXPECT` macros in tests, since they should arrow the tests to proceed (unless the test would otherwise crash just after the failed check).



##########
cpp/src/arrow/type_test.cc:
##########
@@ -364,152 +365,344 @@ TEST(TestField, TestMerge) {
   }
 }
 
-TEST(TestFieldPath, Basics) {
-  auto f0 = field("alpha", int32());
-  auto f1 = field("beta", int32());
-  auto f2 = field("alpha", int32());
-  auto f3 = field("beta", int32());
-  Schema s({f0, f1, f2, f3});
+struct FieldPathTestCase {
+  struct OutputValues {
+    explicit OutputValues(std::vector<int> indices = {})
+        : path(FieldPath(std::move(indices))) {}
 
-  // retrieving a field with single-element FieldPath is equivalent to Schema::field
-  for (int index = 0; index < s.num_fields(); ++index) {
-    ASSERT_OK_AND_EQ(s.field(index), FieldPath({index}).Get(s));
+    template <typename T>
+    auto&& Get() const;
+
+    FieldPath path;
+    std::shared_ptr<Field> field;
+    std::shared_ptr<Array> array;
+    std::shared_ptr<ChunkedArray> chunked_array;
+  };
+
+  static constexpr int kNumColumns = 2;
+  static constexpr int kNumRows = 100;
+  static constexpr int kRandomSeed = 0xbeef;
+
+  // Input for the FieldPath::Get functions in multiple forms
+  std::shared_ptr<Schema> schema;
+  std::shared_ptr<DataType> type;
+  std::shared_ptr<Array> array;
+  std::shared_ptr<RecordBatch> record_batch;
+  std::shared_ptr<ChunkedArray> chunked_array;
+  std::shared_ptr<Table> table;
+
+  template <typename T>
+  auto&& GetInput() const;
+
+  // Number of chunks for each column in the input Table
+  const std::array<int, kNumColumns> num_column_chunks = {15, 20};
+  // Number of chunks in the input ChunkedArray
+  const int num_chunks = 15;
+
+  // Expected outputs for each child;
+  OutputValues v0{{0}}, v1{{1}};
+  OutputValues v1_0{{1, 0}}, v1_1{{1, 1}};
+  OutputValues v1_1_0{{1, 1, 0}}, v1_1_1{{1, 1, 1}};
+  // Expected outputs for nested children with null flattening applied
+  OutputValues v1_0_flat{{1, 0}}, v1_1_flat{{1, 1}};
+  OutputValues v1_1_0_flat{{1, 1, 0}}, v1_1_1_flat{{1, 1, 1}};
+
+  static const FieldPathTestCase* Instance() {
+    static const auto maybe_instance = Make();
+    return &maybe_instance.ValueOrDie();
   }
-  EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid,
-                                  testing::HasSubstr("empty indices cannot be traversed"),
-                                  FieldPath().Get(s));
-  EXPECT_RAISES_WITH_MESSAGE_THAT(IndexError, testing::HasSubstr("index out of range"),
-                                  FieldPath({s.num_fields() * 2}).Get(s));
-}
-
-TEST(TestFieldPath, GetForTable) {
-  using testing::HasSubstr;
-
-  constexpr int kNumRows = 4;
-  auto f0 = field("a", int32());
-  auto f1 = field("b", int32());
-  auto f2 = field("c", struct_({f1}));
-  auto f3 = field("d", struct_({f0, f2}));
-  auto table_schema = schema({f0, f1, f2, f3});
-
-  // Each column has a different chunking
-  ChunkedArrayVector columns(4);
-  columns[0] = ChunkedArrayFromJSON(f0->type(), {"[0,1,2,3]"});
-  columns[1] = ChunkedArrayFromJSON(f1->type(), {"[3,2,1]", "[0]"});
-  columns[2] =
-      ChunkedArrayFromJSON(f2->type(), {R"([{"b":3},{"b":2}])", R"([{"b":1},{"b":0}])"});
-  columns[3] = ChunkedArrayFromJSON(
-      f3->type(), {R"([{"a":0,"c":{"b":3}},{"a":1,"c":{"b":2}}])",
-                   R"([{"a":2,"c":{"b":1}}])", R"([{"a":3,"c":{"b":0}}])"});
-  auto table = Table::Make(table_schema, columns, kNumRows);
-  ASSERT_OK(table->ValidateFull());
 
-  ASSERT_OK_AND_ASSIGN(auto v0, FieldPath({0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v1, FieldPath({1}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v2, FieldPath({2}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v2_0, FieldPath({2, 0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3, FieldPath({3}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_0, FieldPath({3, 0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_1, FieldPath({3, 1}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_1_0, FieldPath({3, 1, 0}).Get(*table));
-
-  EXPECT_EQ(v0->num_chunks(), columns[0]->num_chunks());
-  EXPECT_EQ(v1->num_chunks(), columns[1]->num_chunks());
-  EXPECT_EQ(v2->num_chunks(), columns[2]->num_chunks());
-  EXPECT_EQ(v2_0->num_chunks(), columns[2]->num_chunks());
-  EXPECT_EQ(v3->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_0->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_1->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_1_0->num_chunks(), columns[3]->num_chunks());
-
-  EXPECT_TRUE(columns[0]->Equals(v0));
-  EXPECT_TRUE(columns[0]->Equals(v3_0));
-
-  EXPECT_TRUE(columns[1]->Equals(v1));
-  EXPECT_TRUE(columns[1]->Equals(v2_0));
-  EXPECT_TRUE(columns[1]->Equals(v3_1_0));
-
-  EXPECT_TRUE(columns[2]->Equals(v2));
-  EXPECT_TRUE(columns[2]->Equals(v3_1));
-
-  EXPECT_TRUE(columns[3]->Equals(v3));
-
-  for (const auto& path :
-       {FieldPath({4, 1, 0}), FieldPath({3, 2, 0}), FieldPath{3, 1, 1}}) {
-    EXPECT_RAISES_WITH_MESSAGE_THAT(IndexError, HasSubstr("index out of range"),
-                                    path.Get(*table));
+  static Result<FieldPathTestCase> Make() {
+    // Generate test input based on a single schema. First by creating a StructArray,
+    // then deriving the other input types (ChunkedArray, RecordBatch, Table, etc) from
+    // it. We also compute the expected outputs for each child individually (for each
+    // output type).
+    FieldPathTestCase out;
+    random::RandomArrayGenerator gen(kRandomSeed);
+
+    // Define child fields and input schema
+    out.v1_1_1.field = field("b", boolean());
+    out.v1_1_0.field = field("f", float32());
+    out.v1_1.field = field("s1", struct_({out.v1_1_0.field, out.v1_1_1.field}));
+    out.v1_0.field = field("i", int32());
+    out.v1.field = field("s0", struct_({out.v1_0.field, out.v1_1.field}));
+    out.v0.field = field("u", utf8());
+    out.schema = arrow::schema({out.v0.field, out.v1.field});
+    out.type = struct_(out.schema->fields());
+
+    // Create null bitmaps for the struct fields independent of its childrens'
+    // bitmaps. For FieldPath::GetFlattened, parent/child bitmaps should be combined
+    // - for FieldPath::Get, higher-level nulls are ignored.
+    auto bitmap1_1 = gen.NullBitmap(kNumRows, 0.15);
+    auto bitmap1 = gen.NullBitmap(kNumRows, 0.30);
+
+    // Generate raw leaf arrays
+    out.v1_1_1.array = gen.ArrayOf(out.v1_1_1.field->type(), kNumRows);
+    out.v1_1_0.array = gen.ArrayOf(out.v1_1_0.field->type(), kNumRows);
+    out.v1_0.array = gen.ArrayOf(out.v1_0.field->type(), kNumRows);
+    out.v0.array = gen.ArrayOf(out.v0.field->type(), kNumRows);
+    // Make struct fields from leaf arrays (we use the custom bitmaps here)
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1.array,
+        StructArray::Make({out.v1_1_0.array, out.v1_1_1.array},
+                          {out.v1_1_0.field, out.v1_1_1.field}, bitmap1_1));
+    ARROW_ASSIGN_OR_RAISE(out.v1.array,
+                          StructArray::Make({out.v1_0.array, out.v1_1.array},
+                                            {out.v1_0.field, out.v1_1.field}, bitmap1));
+
+    // Not used to create the test input, but pre-compute flattened versions of nested
+    // arrays for comparisons in the GetFlattened tests.
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_0_flat.array,
+        checked_pointer_cast<StructArray>(out.v1.array)->GetFlattenedField(0));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_flat.array,
+        checked_pointer_cast<StructArray>(out.v1.array)->GetFlattenedField(1));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_0_flat.array,
+        checked_pointer_cast<StructArray>(out.v1_1_flat.array)->GetFlattenedField(0));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_1_flat.array,
+        checked_pointer_cast<StructArray>(out.v1_1_flat.array)->GetFlattenedField(1));
+    // Sanity check
+    ARROW_CHECK(!out.v1_0_flat.array->Equals(out.v1_0.array));
+    ARROW_CHECK(!out.v1_1_flat.array->Equals(out.v1_1.array));
+    ARROW_CHECK(!out.v1_1_0_flat.array->Equals(out.v1_1_0.array));
+    ARROW_CHECK(!out.v1_1_1_flat.array->Equals(out.v1_1_1.array));
+
+    // Finalize the input Array
+    ARROW_ASSIGN_OR_RAISE(out.array, StructArray::Make({out.v0.array, out.v1.array},
+                                                       {out.v0.field, out.v1.field}));
+    ARROW_RETURN_NOT_OK(out.array->ValidateFull());
+    // Finalize the input RecordBatch
+    ARROW_ASSIGN_OR_RAISE(out.record_batch, RecordBatch::FromStructArray(out.array));
+    ARROW_RETURN_NOT_OK(out.record_batch->ValidateFull());
+    // Finalize the input ChunkedArray
+    out.chunked_array = SliceToChunkedArray(*out.array, out.num_chunks);
+    ARROW_RETURN_NOT_OK(out.chunked_array->ValidateFull());
+
+    // For each expected child array, create a chunked equivalent (we use a different
+    // chunk layout for each top-level column to make the Table test more interesting)
+    for (OutputValues* v :
+         {&out.v0, &out.v1, &out.v1_0, &out.v1_1, &out.v1_1_0, &out.v1_1_1,
+          &out.v1_0_flat, &out.v1_1_flat, &out.v1_1_0_flat, &out.v1_1_1_flat}) {
+      v->chunked_array =
+          SliceToChunkedArray(*v->array, out.num_column_chunks[v->path[0]]);
+    }
+    // Finalize the input Table
+    out.table =
+        Table::Make(out.schema, {out.v0.chunked_array, out.v1.chunked_array}, kNumRows);
+    ARROW_RETURN_NOT_OK(out.table->ValidateFull());
+
+    return std::move(out);
   }
-  EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("empty indices cannot be traversed"),
-                                  FieldPath().Get(*table));
-}
-
-TEST(TestFieldPath, GetForChunkedArray) {
-  using testing::HasSubstr;
-
-  auto f0 = field("a", int32());
-  auto f1 = field("b", int32());
-  auto f2 = field("c", struct_({f1}));
-  auto f3 = field("d", struct_({f0, f2}));
-  auto type = struct_({f0, f1, f3});
-
-  auto column0 = ChunkedArrayFromJSON(f0->type(), {"[0,1,2,3]"});
-  auto column1 = ChunkedArrayFromJSON(f1->type(), {"[3,2,1,0]"});
-  auto column2_1 =
-      ChunkedArrayFromJSON(f2->type(), {R"([{"b":3},{"b":2},{"b":1},{"b":0}])"});
-  auto chunked_array = ChunkedArrayFromJSON(
-      type,
-      {
-          R"([{"a":0,"b":3,"d":{"a":0,"c":{"b":3}}}])",
-          R"([{"a":1,"b":2,"d":{"a":1,"c":{"b":2}}},{"a":2,"b":1,"d":{"a":2,"c":{"b":1}}}])",
-          R"([{"a":3,"b":0,"d":{"a":3,"c":{"b":0}}}])",
-      });
-  ASSERT_OK(chunked_array->ValidateFull());
-
-  ASSERT_OK_AND_ASSIGN(auto v0, FieldPath({0}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v1, FieldPath({1}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_0, FieldPath({2, 0}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_1, FieldPath({2, 1}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_1_0, FieldPath({2, 1, 0}).Get(*chunked_array));
-
-  for (const auto& v : {v0, v1, v2_0, v2_1, v2_1_0}) {
-    EXPECT_EQ(v->num_chunks(), chunked_array->num_chunks());
+
+  static std::shared_ptr<ChunkedArray> SliceToChunkedArray(const Array& array,
+                                                           int num_chunks) {
+    ARROW_CHECK(num_chunks > 0 && array.length() >= num_chunks);
+    ArrayVector chunks;
+    chunks.reserve(num_chunks);
+    for (int64_t inc = array.length() / num_chunks, beg = 0,
+                 end = inc + array.length() % num_chunks;
+         end <= array.length(); beg = end, end += inc) {
+      chunks.push_back(array.SliceSafe(beg, end - beg).ValueOrDie());
+    }
+    ARROW_CHECK_EQ(static_cast<int>(chunks.size()), num_chunks);
+    return ChunkedArray::Make(std::move(chunks)).ValueOrDie();
   }
+};
+
+template <>
+auto&& FieldPathTestCase::GetInput<Schema>() const {
+  return this->schema;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<DataType>() const {
+  return this->type;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<Array>() const {
+  return this->array;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<ArrayData>() const {
+  return this->array->data();
+}
+template <>
+auto&& FieldPathTestCase::GetInput<ChunkedArray>() const {
+  return this->chunked_array;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<RecordBatch>() const {
+  return this->record_batch;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<Table>() const {
+  return this->table;
+}
 
-  EXPECT_TRUE(column0->Equals(v0));
-  EXPECT_TRUE(column0->Equals(v2_0));
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<Field>() const {
+  return this->field;
+}
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<Array>() const {
+  return this->array;
+}
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<ArrayData>() const {
+  return this->array->data();
+}
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<ChunkedArray>() const {
+  return this->chunked_array;
+}
 
-  EXPECT_TRUE(column1->Equals(v1));
-  EXPECT_TRUE(column1->Equals(v2_1_0));
-  EXPECT_FALSE(column1->Equals(v2_1));
+class FieldPathTestFixture : public ::testing::Test {
+ public:
+  FieldPathTestFixture() : case_(FieldPathTestCase::Instance()) {}
 
-  EXPECT_TRUE(column2_1->Equals(v2_1));
+ protected:
+  template <typename T>
+  using OutputType = typename internal::FieldPathGetType<T>::element_type;
 
-  EXPECT_RAISES_WITH_MESSAGE_THAT(NotImplemented,
-                                  HasSubstr("Get child data of non-struct chunked array"),
-                                  FieldPath({0}).Get(*column0));
-}
+  template <typename I>
+  void AssertOutputsEqual(const std::shared_ptr<Field>& expected,
+                          const std::shared_ptr<Field>& actual) const {
+    AssertFieldEqual(*expected, *actual);
+  }
+  template <typename I>
+  void AssertOutputsEqual(const std::shared_ptr<Array>& expected,
+                          const std::shared_ptr<Array>& actual) const {
+    AssertArraysEqual(*expected, *actual);
+  }
+  template <typename I>
+  void AssertOutputsEqual(const std::shared_ptr<ChunkedArray>& expected,
+                          const std::shared_ptr<ChunkedArray>& actual) const {
+    if constexpr (std::is_same_v<I, ChunkedArray>) {
+      EXPECT_EQ(case_->chunked_array->num_chunks(), actual->num_chunks());
+    } else {
+      EXPECT_EQ(expected->num_chunks(), actual->num_chunks());
+    }
+    AssertChunkedEquivalent(*expected, *actual);
+  }
+
+  const FieldPathTestCase* case_;
+};
+
+class TestFieldPath : public FieldPathTestFixture {
+ protected:
+  template <typename I, bool Flattened = false>
+  void TestGetWithInvalidIndex() const {
+    const auto& input = case_->GetInput<I>();
+    for (const auto& path :
+         {FieldPath({2, 1, 0}), FieldPath({1, 2, 0}), FieldPath{1, 1, 2}}) {
+      EXPECT_RAISES_WITH_MESSAGE_THAT(IndexError,
+                                      ::testing::HasSubstr("index out of range"),
+                                      internal::GetChild<Flattened>(*input, path));

Review Comment:
   Since the `IndexError` has a more detailed error message, is it possible to test it precisely at least for a hard-coded case?



##########
cpp/src/arrow/type_test.cc:
##########
@@ -693,25 +851,6 @@ TEST(TestFieldRef, DotPathRoundTrip) {
   check_roundtrip(FieldRef("foo", 1, FieldRef("bar", 2, 3), FieldRef()));
 }
 
-TEST(TestFieldPath, Nested) {
-  auto f0 = field("alpha", int32());
-  auto f1_0 = field("alpha", int32());
-  auto f1 = field("beta", struct_({f1_0}));
-  auto f2_0 = field("alpha", int32());
-  auto f2_1_0 = field("alpha", int32());
-  auto f2_1_1 = field("alpha", int32());
-  auto f2_1 = field("gamma", struct_({f2_1_0, f2_1_1}));
-  auto f2 = field("beta", struct_({f2_0, f2_1}));
-  Schema s({f0, f1, f2});
-
-  // retrieving fields with nested indices
-  EXPECT_EQ(FieldPath({0}).Get(s), f0);
-  EXPECT_EQ(FieldPath({1, 0}).Get(s), f1_0);
-  EXPECT_EQ(FieldPath({2, 0}).Get(s), f2_0);
-  EXPECT_EQ(FieldPath({2, 1, 0}).Get(s), f2_1_0);
-  EXPECT_EQ(FieldPath({2, 1, 1}).Get(s), f2_1_1);
-}
-
 TEST(TestFieldRef, Nested) {

Review Comment:
   It seems quite a lot of effort was expended to test `FieldPath` comprehensively (thanks a lot for that!), but the additional `FieldRef` methods do not seem to be exercised here? (am I mistaken?)



##########
cpp/src/arrow/type_test.cc:
##########
@@ -364,152 +365,344 @@ TEST(TestField, TestMerge) {
   }
 }
 
-TEST(TestFieldPath, Basics) {
-  auto f0 = field("alpha", int32());
-  auto f1 = field("beta", int32());
-  auto f2 = field("alpha", int32());
-  auto f3 = field("beta", int32());
-  Schema s({f0, f1, f2, f3});
+struct FieldPathTestCase {
+  struct OutputValues {
+    explicit OutputValues(std::vector<int> indices = {})
+        : path(FieldPath(std::move(indices))) {}
 
-  // retrieving a field with single-element FieldPath is equivalent to Schema::field
-  for (int index = 0; index < s.num_fields(); ++index) {
-    ASSERT_OK_AND_EQ(s.field(index), FieldPath({index}).Get(s));
+    template <typename T>
+    auto&& Get() const;
+
+    FieldPath path;
+    std::shared_ptr<Field> field;
+    std::shared_ptr<Array> array;
+    std::shared_ptr<ChunkedArray> chunked_array;
+  };
+
+  static constexpr int kNumColumns = 2;
+  static constexpr int kNumRows = 100;
+  static constexpr int kRandomSeed = 0xbeef;
+
+  // Input for the FieldPath::Get functions in multiple forms
+  std::shared_ptr<Schema> schema;
+  std::shared_ptr<DataType> type;
+  std::shared_ptr<Array> array;
+  std::shared_ptr<RecordBatch> record_batch;
+  std::shared_ptr<ChunkedArray> chunked_array;
+  std::shared_ptr<Table> table;
+
+  template <typename T>
+  auto&& GetInput() const;
+
+  // Number of chunks for each column in the input Table
+  const std::array<int, kNumColumns> num_column_chunks = {15, 20};
+  // Number of chunks in the input ChunkedArray
+  const int num_chunks = 15;
+
+  // Expected outputs for each child;
+  OutputValues v0{{0}}, v1{{1}};
+  OutputValues v1_0{{1, 0}}, v1_1{{1, 1}};
+  OutputValues v1_1_0{{1, 1, 0}}, v1_1_1{{1, 1, 1}};
+  // Expected outputs for nested children with null flattening applied
+  OutputValues v1_0_flat{{1, 0}}, v1_1_flat{{1, 1}};
+  OutputValues v1_1_0_flat{{1, 1, 0}}, v1_1_1_flat{{1, 1, 1}};
+
+  static const FieldPathTestCase* Instance() {
+    static const auto maybe_instance = Make();
+    return &maybe_instance.ValueOrDie();
   }
-  EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid,
-                                  testing::HasSubstr("empty indices cannot be traversed"),
-                                  FieldPath().Get(s));
-  EXPECT_RAISES_WITH_MESSAGE_THAT(IndexError, testing::HasSubstr("index out of range"),
-                                  FieldPath({s.num_fields() * 2}).Get(s));
-}
-
-TEST(TestFieldPath, GetForTable) {
-  using testing::HasSubstr;
-
-  constexpr int kNumRows = 4;
-  auto f0 = field("a", int32());
-  auto f1 = field("b", int32());
-  auto f2 = field("c", struct_({f1}));
-  auto f3 = field("d", struct_({f0, f2}));
-  auto table_schema = schema({f0, f1, f2, f3});
-
-  // Each column has a different chunking
-  ChunkedArrayVector columns(4);
-  columns[0] = ChunkedArrayFromJSON(f0->type(), {"[0,1,2,3]"});
-  columns[1] = ChunkedArrayFromJSON(f1->type(), {"[3,2,1]", "[0]"});
-  columns[2] =
-      ChunkedArrayFromJSON(f2->type(), {R"([{"b":3},{"b":2}])", R"([{"b":1},{"b":0}])"});
-  columns[3] = ChunkedArrayFromJSON(
-      f3->type(), {R"([{"a":0,"c":{"b":3}},{"a":1,"c":{"b":2}}])",
-                   R"([{"a":2,"c":{"b":1}}])", R"([{"a":3,"c":{"b":0}}])"});
-  auto table = Table::Make(table_schema, columns, kNumRows);
-  ASSERT_OK(table->ValidateFull());
 
-  ASSERT_OK_AND_ASSIGN(auto v0, FieldPath({0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v1, FieldPath({1}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v2, FieldPath({2}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v2_0, FieldPath({2, 0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3, FieldPath({3}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_0, FieldPath({3, 0}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_1, FieldPath({3, 1}).Get(*table));
-  ASSERT_OK_AND_ASSIGN(auto v3_1_0, FieldPath({3, 1, 0}).Get(*table));
-
-  EXPECT_EQ(v0->num_chunks(), columns[0]->num_chunks());
-  EXPECT_EQ(v1->num_chunks(), columns[1]->num_chunks());
-  EXPECT_EQ(v2->num_chunks(), columns[2]->num_chunks());
-  EXPECT_EQ(v2_0->num_chunks(), columns[2]->num_chunks());
-  EXPECT_EQ(v3->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_0->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_1->num_chunks(), columns[3]->num_chunks());
-  EXPECT_EQ(v3_1_0->num_chunks(), columns[3]->num_chunks());
-
-  EXPECT_TRUE(columns[0]->Equals(v0));
-  EXPECT_TRUE(columns[0]->Equals(v3_0));
-
-  EXPECT_TRUE(columns[1]->Equals(v1));
-  EXPECT_TRUE(columns[1]->Equals(v2_0));
-  EXPECT_TRUE(columns[1]->Equals(v3_1_0));
-
-  EXPECT_TRUE(columns[2]->Equals(v2));
-  EXPECT_TRUE(columns[2]->Equals(v3_1));
-
-  EXPECT_TRUE(columns[3]->Equals(v3));
-
-  for (const auto& path :
-       {FieldPath({4, 1, 0}), FieldPath({3, 2, 0}), FieldPath{3, 1, 1}}) {
-    EXPECT_RAISES_WITH_MESSAGE_THAT(IndexError, HasSubstr("index out of range"),
-                                    path.Get(*table));
+  static Result<FieldPathTestCase> Make() {
+    // Generate test input based on a single schema. First by creating a StructArray,
+    // then deriving the other input types (ChunkedArray, RecordBatch, Table, etc) from
+    // it. We also compute the expected outputs for each child individually (for each
+    // output type).
+    FieldPathTestCase out;
+    random::RandomArrayGenerator gen(kRandomSeed);
+
+    // Define child fields and input schema
+    out.v1_1_1.field = field("b", boolean());
+    out.v1_1_0.field = field("f", float32());
+    out.v1_1.field = field("s1", struct_({out.v1_1_0.field, out.v1_1_1.field}));
+    out.v1_0.field = field("i", int32());
+    out.v1.field = field("s0", struct_({out.v1_0.field, out.v1_1.field}));
+    out.v0.field = field("u", utf8());
+    out.schema = arrow::schema({out.v0.field, out.v1.field});
+    out.type = struct_(out.schema->fields());
+
+    // Create null bitmaps for the struct fields independent of its childrens'
+    // bitmaps. For FieldPath::GetFlattened, parent/child bitmaps should be combined
+    // - for FieldPath::Get, higher-level nulls are ignored.
+    auto bitmap1_1 = gen.NullBitmap(kNumRows, 0.15);
+    auto bitmap1 = gen.NullBitmap(kNumRows, 0.30);
+
+    // Generate raw leaf arrays
+    out.v1_1_1.array = gen.ArrayOf(out.v1_1_1.field->type(), kNumRows);
+    out.v1_1_0.array = gen.ArrayOf(out.v1_1_0.field->type(), kNumRows);
+    out.v1_0.array = gen.ArrayOf(out.v1_0.field->type(), kNumRows);
+    out.v0.array = gen.ArrayOf(out.v0.field->type(), kNumRows);
+    // Make struct fields from leaf arrays (we use the custom bitmaps here)
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1.array,
+        StructArray::Make({out.v1_1_0.array, out.v1_1_1.array},
+                          {out.v1_1_0.field, out.v1_1_1.field}, bitmap1_1));
+    ARROW_ASSIGN_OR_RAISE(out.v1.array,
+                          StructArray::Make({out.v1_0.array, out.v1_1.array},
+                                            {out.v1_0.field, out.v1_1.field}, bitmap1));
+
+    // Not used to create the test input, but pre-compute flattened versions of nested
+    // arrays for comparisons in the GetFlattened tests.
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_0_flat.array,
+        checked_pointer_cast<StructArray>(out.v1.array)->GetFlattenedField(0));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_flat.array,
+        checked_pointer_cast<StructArray>(out.v1.array)->GetFlattenedField(1));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_0_flat.array,
+        checked_pointer_cast<StructArray>(out.v1_1_flat.array)->GetFlattenedField(0));
+    ARROW_ASSIGN_OR_RAISE(
+        out.v1_1_1_flat.array,
+        checked_pointer_cast<StructArray>(out.v1_1_flat.array)->GetFlattenedField(1));
+    // Sanity check
+    ARROW_CHECK(!out.v1_0_flat.array->Equals(out.v1_0.array));
+    ARROW_CHECK(!out.v1_1_flat.array->Equals(out.v1_1.array));
+    ARROW_CHECK(!out.v1_1_0_flat.array->Equals(out.v1_1_0.array));
+    ARROW_CHECK(!out.v1_1_1_flat.array->Equals(out.v1_1_1.array));
+
+    // Finalize the input Array
+    ARROW_ASSIGN_OR_RAISE(out.array, StructArray::Make({out.v0.array, out.v1.array},
+                                                       {out.v0.field, out.v1.field}));
+    ARROW_RETURN_NOT_OK(out.array->ValidateFull());
+    // Finalize the input RecordBatch
+    ARROW_ASSIGN_OR_RAISE(out.record_batch, RecordBatch::FromStructArray(out.array));
+    ARROW_RETURN_NOT_OK(out.record_batch->ValidateFull());
+    // Finalize the input ChunkedArray
+    out.chunked_array = SliceToChunkedArray(*out.array, out.num_chunks);
+    ARROW_RETURN_NOT_OK(out.chunked_array->ValidateFull());
+
+    // For each expected child array, create a chunked equivalent (we use a different
+    // chunk layout for each top-level column to make the Table test more interesting)
+    for (OutputValues* v :
+         {&out.v0, &out.v1, &out.v1_0, &out.v1_1, &out.v1_1_0, &out.v1_1_1,
+          &out.v1_0_flat, &out.v1_1_flat, &out.v1_1_0_flat, &out.v1_1_1_flat}) {
+      v->chunked_array =
+          SliceToChunkedArray(*v->array, out.num_column_chunks[v->path[0]]);
+    }
+    // Finalize the input Table
+    out.table =
+        Table::Make(out.schema, {out.v0.chunked_array, out.v1.chunked_array}, kNumRows);
+    ARROW_RETURN_NOT_OK(out.table->ValidateFull());
+
+    return std::move(out);
   }
-  EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("empty indices cannot be traversed"),
-                                  FieldPath().Get(*table));
-}
-
-TEST(TestFieldPath, GetForChunkedArray) {
-  using testing::HasSubstr;
-
-  auto f0 = field("a", int32());
-  auto f1 = field("b", int32());
-  auto f2 = field("c", struct_({f1}));
-  auto f3 = field("d", struct_({f0, f2}));
-  auto type = struct_({f0, f1, f3});
-
-  auto column0 = ChunkedArrayFromJSON(f0->type(), {"[0,1,2,3]"});
-  auto column1 = ChunkedArrayFromJSON(f1->type(), {"[3,2,1,0]"});
-  auto column2_1 =
-      ChunkedArrayFromJSON(f2->type(), {R"([{"b":3},{"b":2},{"b":1},{"b":0}])"});
-  auto chunked_array = ChunkedArrayFromJSON(
-      type,
-      {
-          R"([{"a":0,"b":3,"d":{"a":0,"c":{"b":3}}}])",
-          R"([{"a":1,"b":2,"d":{"a":1,"c":{"b":2}}},{"a":2,"b":1,"d":{"a":2,"c":{"b":1}}}])",
-          R"([{"a":3,"b":0,"d":{"a":3,"c":{"b":0}}}])",
-      });
-  ASSERT_OK(chunked_array->ValidateFull());
-
-  ASSERT_OK_AND_ASSIGN(auto v0, FieldPath({0}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v1, FieldPath({1}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_0, FieldPath({2, 0}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_1, FieldPath({2, 1}).Get(*chunked_array));
-  ASSERT_OK_AND_ASSIGN(auto v2_1_0, FieldPath({2, 1, 0}).Get(*chunked_array));
-
-  for (const auto& v : {v0, v1, v2_0, v2_1, v2_1_0}) {
-    EXPECT_EQ(v->num_chunks(), chunked_array->num_chunks());
+
+  static std::shared_ptr<ChunkedArray> SliceToChunkedArray(const Array& array,
+                                                           int num_chunks) {
+    ARROW_CHECK(num_chunks > 0 && array.length() >= num_chunks);
+    ArrayVector chunks;
+    chunks.reserve(num_chunks);
+    for (int64_t inc = array.length() / num_chunks, beg = 0,
+                 end = inc + array.length() % num_chunks;
+         end <= array.length(); beg = end, end += inc) {
+      chunks.push_back(array.SliceSafe(beg, end - beg).ValueOrDie());
+    }
+    ARROW_CHECK_EQ(static_cast<int>(chunks.size()), num_chunks);
+    return ChunkedArray::Make(std::move(chunks)).ValueOrDie();
   }
+};
+
+template <>
+auto&& FieldPathTestCase::GetInput<Schema>() const {
+  return this->schema;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<DataType>() const {
+  return this->type;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<Array>() const {
+  return this->array;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<ArrayData>() const {
+  return this->array->data();
+}
+template <>
+auto&& FieldPathTestCase::GetInput<ChunkedArray>() const {
+  return this->chunked_array;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<RecordBatch>() const {
+  return this->record_batch;
+}
+template <>
+auto&& FieldPathTestCase::GetInput<Table>() const {
+  return this->table;
+}
 
-  EXPECT_TRUE(column0->Equals(v0));
-  EXPECT_TRUE(column0->Equals(v2_0));
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<Field>() const {
+  return this->field;
+}
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<Array>() const {
+  return this->array;
+}
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<ArrayData>() const {
+  return this->array->data();
+}
+template <>
+auto&& FieldPathTestCase::OutputValues::Get<ChunkedArray>() const {
+  return this->chunked_array;
+}
 
-  EXPECT_TRUE(column1->Equals(v1));
-  EXPECT_TRUE(column1->Equals(v2_1_0));
-  EXPECT_FALSE(column1->Equals(v2_1));
+class FieldPathTestFixture : public ::testing::Test {
+ public:
+  FieldPathTestFixture() : case_(FieldPathTestCase::Instance()) {}
 
-  EXPECT_TRUE(column2_1->Equals(v2_1));
+ protected:
+  template <typename T>
+  using OutputType = typename internal::FieldPathGetType<T>::element_type;
 
-  EXPECT_RAISES_WITH_MESSAGE_THAT(NotImplemented,
-                                  HasSubstr("Get child data of non-struct chunked array"),
-                                  FieldPath({0}).Get(*column0));
-}
+  template <typename I>
+  void AssertOutputsEqual(const std::shared_ptr<Field>& expected,
+                          const std::shared_ptr<Field>& actual) const {
+    AssertFieldEqual(*expected, *actual);
+  }
+  template <typename I>
+  void AssertOutputsEqual(const std::shared_ptr<Array>& expected,
+                          const std::shared_ptr<Array>& actual) const {
+    AssertArraysEqual(*expected, *actual);
+  }
+  template <typename I>
+  void AssertOutputsEqual(const std::shared_ptr<ChunkedArray>& expected,
+                          const std::shared_ptr<ChunkedArray>& actual) const {
+    if constexpr (std::is_same_v<I, ChunkedArray>) {

Review Comment:
   Can you comment to explain this to the reader?



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

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

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