You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by pa...@apache.org on 2023/06/06 15:20:34 UTC

[arrow-nanoarrow] branch main updated: refactor(extensions/nanoarrow_ipc): Reconfigure assembling arrays for better validation (#209)

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

paleolimbot pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-nanoarrow.git


The following commit(s) were added to refs/heads/main by this push:
     new 47a4ef4  refactor(extensions/nanoarrow_ipc): Reconfigure assembling arrays for better validation (#209)
47a4ef4 is described below

commit 47a4ef4113d6539db14713f8696b01012e0022df
Author: Dewey Dunnington <de...@dunnington.ca>
AuthorDate: Tue Jun 6 11:20:29 2023 -0400

    refactor(extensions/nanoarrow_ipc): Reconfigure assembling arrays for better validation (#209)
    
    Currently the decoder always assembles an `ArrowArray` and subsequently
    validates that array. That caused a problem when validating a union
    because the `ArrowArray` builder doesn't have a place to put custom type
    IDs. Also, this is every so slightly wasteful: the `ArrowArray`
    validation allocates an unnecessary `ArrowArrayView` and validates that.
    We already have an `ArrowArrayView` to help with the buffer shuffling,
    so this PR just swaps the direction: the first step is to create the
    `ArrowArrayView`; the second step is to validate the `ArrowArrayView`,
    then populate the `ArrowArray` (if requested).
    
    Closes #178 by erroring if there is insufficient information to validate
    a union array. I don't think this will cause problems with existing code
    that builds a union array but if it does it could be relaxed to the
    previous state (silently ignoring union arrays that can't be validated).
---
 dist/nanoarrow.c                                   |  15 +-
 .../nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc.h    |  39 +--
 .../src/nanoarrow/nanoarrow_ipc_decoder.c          | 295 +++++++++++++++------
 .../src/nanoarrow/nanoarrow_ipc_decoder_test.cc    |  61 ++---
 .../src/nanoarrow/nanoarrow_ipc_reader.c           |  15 +-
 src/nanoarrow/array.c                              |  23 +-
 src/nanoarrow/array_test.cc                        |   9 +-
 src/nanoarrow/nanoarrow.h                          |  10 +
 8 files changed, 302 insertions(+), 165 deletions(-)

diff --git a/dist/nanoarrow.c b/dist/nanoarrow.c
index e5a256d..2aa9159 100644
--- a/dist/nanoarrow.c
+++ b/dist/nanoarrow.c
@@ -1898,9 +1898,9 @@ ArrowErrorCode ArrowArrayInitFromType(struct ArrowArray* array,
   return NANOARROW_OK;
 }
 
-static ArrowErrorCode ArrowArrayInitFromArrayView(struct ArrowArray* array,
-                                                  struct ArrowArrayView* array_view,
-                                                  struct ArrowError* error) {
+ArrowErrorCode ArrowArrayInitFromArrayView(struct ArrowArray* array,
+                                           struct ArrowArrayView* array_view,
+                                           struct ArrowError* error) {
   ArrowArrayInitFromType(array, array_view->storage_type);
   struct ArrowArrayPrivateData* private_data =
       (struct ArrowArrayPrivateData*)array->private_data;
@@ -2670,8 +2670,8 @@ ArrowErrorCode ArrowArrayViewValidateFull(struct ArrowArrayView* array_view,
     } else if (_ArrowParsedUnionTypeIdsWillEqualChildIndices(
                    array_view->union_type_id_map, array_view->n_children,
                    array_view->n_children)) {
-      NANOARROW_RETURN_NOT_OK(ArrowAssertRangeInt8(array_view->buffer_views[0], 0,
-                                                   (int8_t)(array_view->n_children - 1), error));
+      NANOARROW_RETURN_NOT_OK(ArrowAssertRangeInt8(
+          array_view->buffer_views[0], 0, (int8_t)(array_view->n_children - 1), error));
     } else {
       NANOARROW_RETURN_NOT_OK(ArrowAssertInt8In(array_view->buffer_views[0],
                                                 array_view->union_type_id_map + 128,
@@ -2793,8 +2793,9 @@ static void ArrowBasicArrayStreamRelease(struct ArrowArrayStream* array_stream)
 
 ArrowErrorCode ArrowBasicArrayStreamInit(struct ArrowArrayStream* array_stream,
                                          struct ArrowSchema* schema, int64_t n_arrays) {
-  struct BasicArrayStreamPrivate* private_data = (struct BasicArrayStreamPrivate*)ArrowMalloc(
-      sizeof(struct BasicArrayStreamPrivate));
+  struct BasicArrayStreamPrivate* private_data =
+      (struct BasicArrayStreamPrivate*)ArrowMalloc(
+          sizeof(struct BasicArrayStreamPrivate));
   if (private_data == NULL) {
     return ENOMEM;
   }
diff --git a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc.h b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc.h
index 5747cec..4078708 100644
--- a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc.h
+++ b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc.h
@@ -39,12 +39,12 @@
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowIpcDecoderDecodeHeader)
 #define ArrowIpcDecoderDecodeSchema \
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowIpcDecoderDecodeSchema)
+#define ArrowIpcDecoderDecodeArrayView \
+  NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowIpcDecoderDecodeArrayView)
 #define ArrowIpcDecoderDecodeArray \
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowIpcDecoderDecodeArray)
 #define ArrowIpcDecoderDecodeArrayFromShared \
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowIpcDecoderDecodeArrayFromShared)
-#define ArrowIpcDecoderValidateArray \
-  NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowIpcDecoderValidateArray)
 #define ArrowIpcDecoderSetSchema \
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowIpcDecoderSetSchema)
 #define ArrowIpcDecoderSetEndianness \
@@ -267,6 +267,23 @@ ArrowErrorCode ArrowIpcDecoderSetSchema(struct ArrowIpcDecoder* decoder,
 ArrowErrorCode ArrowIpcDecoderSetEndianness(struct ArrowIpcDecoder* decoder,
                                             enum ArrowIpcEndianness endianness);
 
+/// \brief Decode an ArrowArrayView
+///
+/// After a successful call to ArrowIpcDecoderDecodeHeader(), deserialize the content
+/// of body into an internally-managed ArrowArrayView and return it. Note that field index
+/// does not equate to column index if any columns contain nested types. Use a value of -1
+/// to decode the entire array into a struct. The pointed-to ArrowArrayView is owned by
+/// the ArrowIpcDecoder and must not be released.
+///
+/// For streams that match system endianness and do not use compression, this operation
+/// will not perform any heap allocations; however, the buffers referred to by the
+/// returned ArrowArrayView are only valid as long as the buffer referred to by body stays
+/// valid.
+ArrowErrorCode ArrowIpcDecoderDecodeArrayView(struct ArrowIpcDecoder* decoder,
+                                              struct ArrowBufferView body, int64_t i,
+                                              struct ArrowArrayView** out,
+                                              struct ArrowError* error);
+
 /// \brief Decode an ArrowArray
 ///
 /// After a successful call to ArrowIpcDecoderDecodeHeader(), assemble an ArrowArray given
@@ -281,6 +298,7 @@ ArrowErrorCode ArrowIpcDecoderSetEndianness(struct ArrowIpcDecoder* decoder,
 ArrowErrorCode ArrowIpcDecoderDecodeArray(struct ArrowIpcDecoder* decoder,
                                           struct ArrowBufferView body, int64_t i,
                                           struct ArrowArray* out,
+                                          enum ArrowValidationLevel validation_level,
                                           struct ArrowError* error);
 
 /// \brief Decode an ArrowArray from an owned buffer
@@ -290,19 +308,10 @@ ArrowErrorCode ArrowIpcDecoderDecodeArray(struct ArrowIpcDecoder* decoder,
 /// more calls to ArrowIpcDecoderDecodeArrayFromShared(). If
 /// ArrowIpcSharedBufferIsThreadSafe() returns 0, out must not be released by another
 /// thread.
-ArrowErrorCode ArrowIpcDecoderDecodeArrayFromShared(struct ArrowIpcDecoder* decoder,
-                                                    struct ArrowIpcSharedBuffer* shared,
-                                                    int64_t i, struct ArrowArray* out,
-                                                    struct ArrowError* error);
-
-/// \brief Validate a decoded ArrowArray
-///
-/// Verifies buffer lengths and contents depending on validation_level. Users
-/// are reccomended to use NANOARROW_VALIDATION_LEVEL_FULL as any lesser value
-/// may result in some types of corrupted data crashing a process on read.
-ArrowErrorCode ArrowIpcDecoderValidateArray(struct ArrowArray* decoded,
-                                            enum ArrowValidationLevel validation_level,
-                                            struct ArrowError* error);
+ArrowErrorCode ArrowIpcDecoderDecodeArrayFromShared(
+    struct ArrowIpcDecoder* decoder, struct ArrowIpcSharedBuffer* shared, int64_t i,
+    struct ArrowArray* out, enum ArrowValidationLevel validation_level,
+    struct ArrowError* error);
 
 /// \brief An user-extensible input data source
 struct ArrowIpcInputStream {
diff --git a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder.c b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder.c
index e401cac..571dd79 100644
--- a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder.c
+++ b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder.c
@@ -46,18 +46,44 @@
 #include "nanoarrow_ipc.h"
 #include "nanoarrow_ipc_flatcc_generated.h"
 
+// Internal representation of a parsed "Field" from flatbuffers. This
+// represents a field in a depth-first walk of column arrays and their
+// children.
 struct ArrowIpcField {
+  // Pointer to the ArrowIpcDecoderPrivate::array_view or child for this node
   struct ArrowArrayView* array_view;
+  // Pointer to the ArrowIpcDecoderPrivate::array or child for this node. This
+  // array is scratch space for any intermediary allocations (i.e., it is never moved
+  // to the user).
+  struct ArrowArray* array;
+  // The cumulative number of buffers preceeding this node.
   int64_t buffer_offset;
 };
 
+// Internal data specific to the read/decode process
 struct ArrowIpcDecoderPrivate {
+  // The endianness that will be assumed for decoding future RecordBatch messages
   enum ArrowIpcEndianness endianness;
+  // A cached system endianness value
   enum ArrowIpcEndianness system_endianness;
+  // An ArrowArrayView whose length/null_count/buffers are set directly from the
+  // deserialized flatbuffer message (i.e., no fully underlying ArrowArray exists,
+  // although some buffers may be temporarily owned by ArrowIpcDecoderPrivate::array).
   struct ArrowArrayView array_view;
+  // An ArrowArray with the same structure as the ArrowArrayView whose ArrowArrayBuffer()
+  // values are used to allocate or store memory when this is required. This ArrowArray
+  // is never moved to the caller; however, its buffers may be moved to the final output
+  // ArrowArray if the caller requests one.
+  struct ArrowArray array;
+  // The number of fields in the flattened depth-first walk of columns and their children
   int64_t n_fields;
+  // Array of cached information such that given a field index it is possible to locate
+  // the ArrowArrayView/ArrowArray where the depth-first buffer/field walk should start.
   struct ArrowIpcField* fields;
+  // The number of buffers that future RecordBatch messages must have to match the schema
+  // that has been set.
   int64_t n_buffers;
+  // A pointer to the last flatbuffers message.
   const void* last_message;
 };
 
@@ -215,6 +241,10 @@ void ArrowIpcDecoderReset(struct ArrowIpcDecoder* decoder) {
   if (private_data != NULL) {
     ArrowArrayViewReset(&private_data->array_view);
 
+    if (private_data->array.release != NULL) {
+      private_data->array.release(&private_data->array);
+    }
+
     if (private_data->fields != NULL) {
       ArrowFree(private_data->fields);
       private_data->n_fields = 0;
@@ -1097,20 +1127,23 @@ static void ArrowIpcDecoderCountFields(struct ArrowSchema* schema, int64_t* n_fi
 }
 
 static void ArrowIpcDecoderInitFields(struct ArrowIpcField* fields,
-                                      struct ArrowArrayView* view, int64_t* n_fields,
+                                      struct ArrowArrayView* array_view,
+                                      struct ArrowArray* array, int64_t* n_fields,
                                       int64_t* n_buffers) {
   struct ArrowIpcField* field = fields + (*n_fields);
-  field->array_view = view;
+  field->array_view = array_view;
+  field->array = array;
   field->buffer_offset = *n_buffers;
 
   for (int i = 0; i < 3; i++) {
-    *n_buffers += view->layout.buffer_type[i] != NANOARROW_BUFFER_TYPE_NONE;
+    *n_buffers += array_view->layout.buffer_type[i] != NANOARROW_BUFFER_TYPE_NONE;
   }
 
   *n_fields += 1;
 
-  for (int64_t i = 0; i < view->n_children; i++) {
-    ArrowIpcDecoderInitFields(fields, view->children[i], n_fields, n_buffers);
+  for (int64_t i = 0; i < array_view->n_children; i++) {
+    ArrowIpcDecoderInitFields(fields, array_view->children[i], array->children[i],
+                              n_fields, n_buffers);
   }
 }
 
@@ -1124,14 +1157,19 @@ ArrowErrorCode ArrowIpcDecoderSetSchema(struct ArrowIpcDecoder* decoder,
   private_data->n_buffers = 0;
   private_data->n_fields = 0;
   ArrowArrayViewReset(&private_data->array_view);
+  if (private_data->array.release != NULL) {
+    private_data->array.release(&private_data->array);
+  }
   if (private_data->fields != NULL) {
     ArrowFree(private_data->fields);
   }
 
-  // Allocate Array and ArrayView based on schema without moving the schema
-  // this will fail if the schema is not valid.
+  // Allocate Array and ArrayView based on schema without moving the schema.
+  // This will fail if the schema is not valid.
   NANOARROW_RETURN_NOT_OK(
       ArrowArrayViewInitFromSchema(&private_data->array_view, schema, error));
+  NANOARROW_RETURN_NOT_OK(ArrowArrayInitFromArrayView(&private_data->array,
+                                                      &private_data->array_view, error));
 
   // Root must be a struct
   if (private_data->array_view.storage_type != NANOARROW_TYPE_STRUCT) {
@@ -1151,8 +1189,8 @@ ArrowErrorCode ArrowIpcDecoderSetSchema(struct ArrowIpcDecoder* decoder,
 
   // Init field information and calculate starting buffer offset for each
   int64_t field_i = 0;
-  ArrowIpcDecoderInitFields(private_data->fields, &private_data->array_view, &field_i,
-                            &private_data->n_buffers);
+  ArrowIpcDecoderInitFields(private_data->fields, &private_data->array_view,
+                            &private_data->array, &field_i, &private_data->n_buffers);
 
   return NANOARROW_OK;
 }
@@ -1196,14 +1234,17 @@ struct ArrowIpcBufferSource {
 /// non-owned view of memory that must be copied slice-wise or (2) adding a reference
 /// to an ArrowIpcSharedBuffer and returning a slice of that memory.
 struct ArrowIpcBufferFactory {
-  /// \brief User-defined callback to create initialize the desired buffer into dst
+  /// \brief User-defined callback to populate a buffer view
   ///
   /// At the time that this callback is called, the ArrowIpcBufferSource has been checked
-  /// to ensure that it is within the body size declared by the message header. If
-  /// NANOARROW_OK is returned, the caller is responsible for dst. Otherwise, error must
-  /// contain a null-terminated message.
+  /// to ensure that it is within the body size declared by the message header. A
+  /// possibly preallocated ArrowBuffer (dst) is provided, which implementations must use
+  /// if an allocation is required (in which case the view must be populated pointing to
+  /// the contents of the ArrowBuffer) If NANOARROW_OK is not returned, error must contain
+  /// a null-terminated message.
   ArrowErrorCode (*make_buffer)(struct ArrowIpcBufferFactory* factory,
-                                struct ArrowIpcBufferSource* src, struct ArrowBuffer* dst,
+                                struct ArrowIpcBufferSource* src,
+                                struct ArrowBufferView* dst_view, struct ArrowBuffer* dst,
                                 struct ArrowError* error);
 
   /// \brief Caller-defined private data to be used in the callback.
@@ -1215,16 +1256,12 @@ struct ArrowIpcBufferFactory {
 
 static ArrowErrorCode ArrowIpcMakeBufferFromView(struct ArrowIpcBufferFactory* factory,
                                                  struct ArrowIpcBufferSource* src,
+                                                 struct ArrowBufferView* dst_view,
                                                  struct ArrowBuffer* dst,
                                                  struct ArrowError* error) {
   struct ArrowBufferView* body = (struct ArrowBufferView*)factory->private_data;
-
-  struct ArrowBufferView view;
-  view.data.as_uint8 = body->data.as_uint8 + src->body_offset_bytes;
-  view.size_bytes = src->buffer_length_bytes;
-
-  ArrowBufferInit(dst);
-  NANOARROW_RETURN_NOT_OK_WITH_ERROR(ArrowBufferAppendBufferView(dst, view), error);
+  dst_view->data.as_uint8 = body->data.as_uint8 + src->body_offset_bytes;
+  dst_view->size_bytes = src->buffer_length_bytes;
   return NANOARROW_OK;
 }
 
@@ -1238,13 +1275,17 @@ static struct ArrowIpcBufferFactory ArrowIpcBufferFactoryFromView(
 
 static ArrowErrorCode ArrowIpcMakeBufferFromShared(struct ArrowIpcBufferFactory* factory,
                                                    struct ArrowIpcBufferSource* src,
+                                                   struct ArrowBufferView* dst_view,
                                                    struct ArrowBuffer* dst,
                                                    struct ArrowError* error) {
   struct ArrowIpcSharedBuffer* shared =
       (struct ArrowIpcSharedBuffer*)factory->private_data;
+  ArrowBufferReset(dst);
   ArrowIpcSharedBufferClone(shared, dst);
   dst->data += src->body_offset_bytes;
   dst->size_bytes = src->buffer_length_bytes;
+  dst_view->data.data = dst->data;
+  dst_view->size_bytes = dst->size_bytes;
   return NANOARROW_OK;
 }
 
@@ -1267,8 +1308,11 @@ struct ArrowIpcArraySetter {
 };
 
 static int ArrowIpcDecoderMakeBuffer(struct ArrowIpcArraySetter* setter, int64_t offset,
-                                     int64_t length, struct ArrowBuffer* out,
-                                     struct ArrowError* error) {
+                                     int64_t length, struct ArrowBufferView* out_view,
+                                     struct ArrowBuffer* out, struct ArrowError* error) {
+  out_view->data.data = NULL;
+  out_view->size_bytes = 0;
+
   if (length == 0) {
     return NANOARROW_OK;
   }
@@ -1299,54 +1343,124 @@ static int ArrowIpcDecoderMakeBuffer(struct ArrowIpcArraySetter* setter, int64_t
   setter->src.body_offset_bytes = offset;
   setter->src.buffer_length_bytes = length;
   NANOARROW_RETURN_NOT_OK(
-      setter->factory.make_buffer(&setter->factory, &setter->src, out, error));
+      setter->factory.make_buffer(&setter->factory, &setter->src, out_view, out, error));
   return NANOARROW_OK;
 }
 
-static int ArrowIpcDecoderWalkGetArray(struct ArrowIpcArraySetter* setter,
-                                       struct ArrowArray* array,
+static int ArrowIpcDecoderWalkGetArray(struct ArrowArrayView* array_view,
+                                       struct ArrowArray* array, struct ArrowArray* out,
                                        struct ArrowError* error) {
+  out->length = array_view->length;
+  out->null_count = array_view->null_count;
+
+  for (int64_t i = 0; i < array->n_buffers; i++) {
+    struct ArrowBufferView view = array_view->buffer_views[i];
+    struct ArrowBuffer* scratch_buffer = ArrowArrayBuffer(array, i);
+    struct ArrowBuffer* buffer_out = ArrowArrayBuffer(out, i);
+
+    // If the scratch buffer was used, move it to the final array. Otherwise,
+    // copy the view.
+    if (scratch_buffer->size_bytes == 0) {
+      NANOARROW_RETURN_NOT_OK(ArrowBufferAppendBufferView(buffer_out, view));
+    } else if (scratch_buffer->data == view.data.as_uint8) {
+      ArrowBufferMove(scratch_buffer, buffer_out);
+    } else {
+      ArrowErrorSet(
+          error,
+          "Internal: scratch buffer was used but doesn't point to the same data as view");
+      return EINVAL;
+    }
+  }
+
+  for (int64_t i = 0; i < array->n_children; i++) {
+    NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderWalkGetArray(
+        array_view->children[i], array->children[i], out->children[i], error));
+  }
+
+  return NANOARROW_OK;
+}
+
+static int ArrowIpcDecoderWalkSetArrayView(struct ArrowIpcArraySetter* setter,
+                                           struct ArrowArrayView* array_view,
+                                           struct ArrowArray* array,
+                                           struct ArrowError* error) {
   ns(FieldNode_struct_t) field =
       ns(FieldNode_vec_at(setter->fields, (size_t)setter->field_i));
-  array->length = ns(FieldNode_length(field));
-  array->null_count = ns(FieldNode_null_count(field));
+  array_view->length = ns(FieldNode_length(field));
+  array_view->null_count = ns(FieldNode_null_count(field));
   setter->field_i += 1;
 
-  for (int64_t i = 0; i < array->n_buffers; i++) {
+  for (int64_t i = 0; i < 3; i++) {
+    if (array_view->layout.buffer_type[i] == NANOARROW_BUFFER_TYPE_NONE) {
+      break;
+    }
+
     ns(Buffer_struct_t) buffer =
         ns(Buffer_vec_at(setter->buffers, (size_t)setter->buffer_i));
     int64_t buffer_offset = ns(Buffer_offset(buffer));
     int64_t buffer_length = ns(Buffer_length(buffer));
     setter->buffer_i += 1;
 
+    // Provide a buffer that will be used if any allocation has to occur
     struct ArrowBuffer* buffer_dst = ArrowArrayBuffer(array, i);
-    NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderMakeBuffer(setter, buffer_offset,
-                                                      buffer_length, buffer_dst, error));
-  }
 
-  for (int64_t i = 0; i < array->n_children; i++) {
+    // Attempt to re-use any previous allocation unless this buffer is
+    // wrapping a custom allocator.
+    if (buffer_dst->allocator.private_data != NULL) {
+      ArrowBufferReset(buffer_dst);
+    } else {
+      buffer_dst->size_bytes = 0;
+    }
+
     NANOARROW_RETURN_NOT_OK(
-        ArrowIpcDecoderWalkGetArray(setter, array->children[i], error));
+        ArrowIpcDecoderMakeBuffer(setter, buffer_offset, buffer_length,
+                                  &array_view->buffer_views[i], buffer_dst, error));
+  }
+
+  for (int64_t i = 0; i < array_view->n_children; i++) {
+    NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderWalkSetArrayView(
+        setter, array_view->children[i], array->children[i], error));
   }
 
   return NANOARROW_OK;
 }
 
-static int ArrowIpcArrayInitFromArrayView(struct ArrowArray* array,
-                                          struct ArrowArrayView* array_view) {
-  NANOARROW_RETURN_NOT_OK(ArrowArrayInitFromType(array, array_view->storage_type));
-  NANOARROW_RETURN_NOT_OK(ArrowArrayAllocateChildren(array, array_view->n_children));
-  for (int64_t i = 0; i < array_view->n_children; i++) {
+static ArrowErrorCode ArrowIpcDecoderDecodeArrayInternal(struct ArrowIpcDecoder* decoder,
+                                                         int64_t field_i,
+                                                         struct ArrowArray* out,
+                                                         struct ArrowError* error) {
+  struct ArrowIpcDecoderPrivate* private_data =
+      (struct ArrowIpcDecoderPrivate*)decoder->private_data;
+
+  struct ArrowIpcField* root = private_data->fields + field_i + 1;
+
+  if (field_i == -1) {
+    NANOARROW_RETURN_NOT_OK(
+        ArrowArrayInitFromArrayView(out, &private_data->array_view, error));
+    out->length = private_data->array_view.length;
+    out->null_count = private_data->array_view.null_count;
+
+    for (int64_t i = 0; i < private_data->array_view.n_children; i++) {
+      NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderWalkGetArray(
+          private_data->array_view.children[i], private_data->array.children[i],
+          out->children[i], error));
+    }
+
+  } else {
+    NANOARROW_RETURN_NOT_OK(ArrowArrayInitFromArrayView(out, root->array_view, error));
     NANOARROW_RETURN_NOT_OK(
-        ArrowIpcArrayInitFromArrayView(array->children[i], array_view->children[i]));
+        ArrowIpcDecoderWalkGetArray(root->array_view, root->array, out, error));
   }
 
+  // If validation is going to happen it has already occurred
+  ArrowArrayFinishBuilding(out, NANOARROW_VALIDATION_LEVEL_NONE, error);
+
   return NANOARROW_OK;
 }
 
-static ArrowErrorCode ArrowIpcDecoderDecodeArrayInternal(
+static ArrowErrorCode ArrowIpcDecoderDecodeArrayViewInternal(
     struct ArrowIpcDecoder* decoder, struct ArrowIpcBufferFactory factory,
-    int64_t field_i, struct ArrowArray* out, struct ArrowError* error) {
+    int64_t field_i, struct ArrowArrayView** out_view, struct ArrowError* error) {
   struct ArrowIpcDecoderPrivate* private_data =
       (struct ArrowIpcDecoderPrivate*)decoder->private_data;
 
@@ -1361,14 +1475,6 @@ static ArrowErrorCode ArrowIpcDecoderDecodeArrayInternal(
   // RecordBatch messages don't count the root node but decoder->fields does
   struct ArrowIpcField* root = private_data->fields + field_i + 1;
 
-  struct ArrowArray temp;
-  temp.release = NULL;
-  int result = ArrowIpcArrayInitFromArrayView(&temp, root->array_view);
-  if (result != NANOARROW_OK) {
-    ArrowErrorSet(error, "Failed to initialize output array");
-    return result;
-  }
-
   struct ArrowIpcArraySetter setter;
   setter.fields = ns(RecordBatch_nodes(batch));
   setter.field_i = field_i;
@@ -1382,56 +1488,75 @@ static ArrowErrorCode ArrowIpcDecoderDecodeArrayInternal(
   // The flatbuffers FieldNode doesn't count the root struct so we have to loop over the
   // children ourselves
   if (field_i == -1) {
-    temp.length = ns(RecordBatch_length(batch));
-    temp.null_count = 0;
+    root->array_view->length = ns(RecordBatch_length(batch));
+    root->array_view->null_count = 0;
     setter.field_i++;
     setter.buffer_i++;
 
-    for (int64_t i = 0; i < temp.n_children; i++) {
-      result = ArrowIpcDecoderWalkGetArray(&setter, temp.children[i], error);
-      if (result != NANOARROW_OK) {
-        temp.release(&temp);
-        return result;
-      }
+    for (int64_t i = 0; i < root->array_view->n_children; i++) {
+      NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderWalkSetArrayView(
+          &setter, root->array_view->children[i], root->array->children[i], error));
     }
   } else {
-    result = ArrowIpcDecoderWalkGetArray(&setter, &temp, error);
-    if (result != NANOARROW_OK) {
-      temp.release(&temp);
-      return result;
-    }
-  }
-
-  // Finish building to flush internal pointers but defer validation to
-  // ArrowIpcDecoderValidateArray()
-  result = ArrowArrayFinishBuilding(&temp, NANOARROW_VALIDATION_LEVEL_NONE, error);
-  if (result != NANOARROW_OK) {
-    temp.release(&temp);
-    return result;
+    NANOARROW_RETURN_NOT_OK(
+        ArrowIpcDecoderWalkSetArrayView(&setter, root->array_view, root->array, error));
   }
 
-  ArrowArrayMove(&temp, out);
+  *out_view = root->array_view;
   return NANOARROW_OK;
 }
 
+ArrowErrorCode ArrowIpcDecoderDecodeArrayView(struct ArrowIpcDecoder* decoder,
+                                              struct ArrowBufferView body, int64_t i,
+                                              struct ArrowArrayView** out,
+                                              struct ArrowError* error) {
+  return ArrowIpcDecoderDecodeArrayViewInternal(
+      decoder, ArrowIpcBufferFactoryFromView(&body), i, out, error);
+}
+
 ArrowErrorCode ArrowIpcDecoderDecodeArray(struct ArrowIpcDecoder* decoder,
                                           struct ArrowBufferView body, int64_t i,
                                           struct ArrowArray* out,
+                                          enum ArrowValidationLevel validation_level,
                                           struct ArrowError* error) {
-  return ArrowIpcDecoderDecodeArrayInternal(decoder, ArrowIpcBufferFactoryFromView(&body),
-                                            i, out, error);
-}
+  struct ArrowArrayView* array_view;
+  NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderDecodeArrayViewInternal(
+      decoder, ArrowIpcBufferFactoryFromView(&body), i, &array_view, error));
+
+  NANOARROW_RETURN_NOT_OK(ArrowArrayViewValidate(array_view, validation_level, error));
 
-ArrowErrorCode ArrowIpcDecoderDecodeArrayFromShared(struct ArrowIpcDecoder* decoder,
-                                                    struct ArrowIpcSharedBuffer* body,
-                                                    int64_t i, struct ArrowArray* out,
-                                                    struct ArrowError* error) {
-  return ArrowIpcDecoderDecodeArrayInternal(
-      decoder, ArrowIpcBufferFactoryFromShared(body), i, out, error);
+  struct ArrowArray temp;
+  temp.release = NULL;
+  int result = ArrowIpcDecoderDecodeArrayInternal(decoder, i, &temp, error);
+  if (result != NANOARROW_OK && temp.release != NULL) {
+    temp.release(&temp);
+  } else if (result != NANOARROW_OK) {
+    return result;
+  }
+
+  ArrowArrayMove(&temp, out);
+  return NANOARROW_OK;
 }
 
-ArrowErrorCode ArrowIpcDecoderValidateArray(struct ArrowArray* decoded,
-                                            enum ArrowValidationLevel validation_level,
-                                            struct ArrowError* error) {
-  return ArrowArrayFinishBuilding(decoded, validation_level, error);
+ArrowErrorCode ArrowIpcDecoderDecodeArrayFromShared(
+    struct ArrowIpcDecoder* decoder, struct ArrowIpcSharedBuffer* body, int64_t i,
+    struct ArrowArray* out, enum ArrowValidationLevel validation_level,
+    struct ArrowError* error) {
+  struct ArrowArrayView* array_view;
+  NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderDecodeArrayViewInternal(
+      decoder, ArrowIpcBufferFactoryFromShared(body), i, &array_view, error));
+
+  NANOARROW_RETURN_NOT_OK(ArrowArrayViewValidate(array_view, validation_level, error));
+
+  struct ArrowArray temp;
+  temp.release = NULL;
+  int result = ArrowIpcDecoderDecodeArrayInternal(decoder, i, &temp, error);
+  if (result != NANOARROW_OK && temp.release != NULL) {
+    temp.release(&temp);
+  } else if (result != NANOARROW_OK) {
+    return result;
+  }
+
+  ArrowArrayMove(&temp, out);
+  return NANOARROW_OK;
 }
diff --git a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder_test.cc b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder_test.cc
index ad989fc..62e4bb0 100644
--- a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder_test.cc
+++ b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_decoder_test.cc
@@ -35,6 +35,7 @@ using namespace arrow;
 extern "C" {
 struct ArrowIpcField {
   struct ArrowArrayView* array_view;
+  struct ArrowArray* array;
   int64_t buffer_offset;
 };
 
@@ -42,6 +43,7 @@ struct ArrowIpcDecoderPrivate {
   enum ArrowIpcEndianness endianness;
   enum ArrowIpcEndianness system_endianness;
   struct ArrowArrayView array_view;
+  struct ArrowArray array;
   int64_t n_fields;
   struct ArrowIpcField* fields;
   int64_t n_buffers;
@@ -274,7 +276,9 @@ TEST(NanoarrowIpcTest, NanoarrowIpcDecodeSimpleRecordBatch) {
       reinterpret_cast<struct ArrowIpcDecoderPrivate*>(decoder.private_data);
 
   // Attempt to get array should fail nicely here
-  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, data, 0, nullptr, &error), EINVAL);
+  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, data, 0, nullptr,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, &error),
+            EINVAL);
   EXPECT_STREQ(error.message, "decoder did not just decode a RecordBatch message");
 
   ASSERT_EQ(ArrowIpcDecoderSetSchema(&decoder, &schema, nullptr), NANOARROW_OK);
@@ -292,11 +296,9 @@ TEST(NanoarrowIpcTest, NanoarrowIpcDecodeSimpleRecordBatch) {
   body.size_bytes = decoder.body_size_bytes;
 
   // Check full struct extract
-  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, -1, &array, nullptr),
+  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, -1, &array,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
             NANOARROW_OK);
-  EXPECT_EQ(
-      ArrowIpcDecoderValidateArray(&array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
-      NANOARROW_OK);
   EXPECT_EQ(array.length, 3);
   EXPECT_EQ(array.null_count, 0);
   ASSERT_EQ(array.n_children, 1);
@@ -310,10 +312,9 @@ TEST(NanoarrowIpcTest, NanoarrowIpcDecodeSimpleRecordBatch) {
   array.release(&array);
 
   // Check field extract
-  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array, nullptr), NANOARROW_OK);
-  EXPECT_EQ(
-      ArrowIpcDecoderValidateArray(&array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
-      NANOARROW_OK);
+  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
+            NANOARROW_OK);
   ASSERT_EQ(array.n_buffers, 2);
   ASSERT_EQ(array.length, 3);
   EXPECT_EQ(array.null_count, 0);
@@ -323,7 +324,9 @@ TEST(NanoarrowIpcTest, NanoarrowIpcDecodeSimpleRecordBatch) {
 
   // Field extract should fail if compression was set
   decoder.codec = NANOARROW_IPC_COMPRESSION_TYPE_ZSTD;
-  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array, &error), ENOTSUP);
+  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, &error),
+            ENOTSUP);
   EXPECT_STREQ(error.message, "The nanoarrow_ipc extension does not support compression");
   decoder.codec = NANOARROW_IPC_COMPRESSION_TYPE_NONE;
 
@@ -333,14 +336,18 @@ TEST(NanoarrowIpcTest, NanoarrowIpcDecodeSimpleRecordBatch) {
   } else {
     ArrowIpcDecoderSetEndianness(&decoder, NANOARROW_IPC_ENDIANNESS_LITTLE);
   }
-  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array, &error), ENOTSUP);
+  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, &error),
+            ENOTSUP);
   EXPECT_STREQ(error.message,
                "The nanoarrow_ipc extension does not support non-system endianness");
   ArrowIpcDecoderSetEndianness(&decoder, NANOARROW_IPC_ENDIANNESS_UNINITIALIZED);
 
   // Field extract should fail if body is too small
   decoder.body_size_bytes = 0;
-  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array, &error), EINVAL);
+  EXPECT_EQ(ArrowIpcDecoderDecodeArray(&decoder, body, 0, &array,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, &error),
+            EINVAL);
   EXPECT_STREQ(error.message, "Buffer requires body offsets [0..12) but body has size 0");
 
   // Should error if the number of buffers or field nodes doesn't match
@@ -481,11 +488,9 @@ TEST(NanoarrowIpcTest, NanoarrowIpcDecodeSimpleRecordBatchFromShared) {
   ASSERT_EQ(ArrowIpcSharedBufferInit(&shared, &body), NANOARROW_OK);
 
   // Check full struct extract
-  EXPECT_EQ(ArrowIpcDecoderDecodeArrayFromShared(&decoder, &shared, -1, &array, nullptr),
+  EXPECT_EQ(ArrowIpcDecoderDecodeArrayFromShared(
+                &decoder, &shared, -1, &array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
             NANOARROW_OK);
-  EXPECT_EQ(
-      ArrowIpcDecoderValidateArray(&array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
-      NANOARROW_OK);
 
   EXPECT_EQ(array.length, 3);
   EXPECT_EQ(array.null_count, 0);
@@ -500,11 +505,9 @@ TEST(NanoarrowIpcTest, NanoarrowIpcDecodeSimpleRecordBatchFromShared) {
   array.release(&array);
 
   // Check field extract
-  EXPECT_EQ(ArrowIpcDecoderDecodeArrayFromShared(&decoder, &shared, 0, &array, nullptr),
+  EXPECT_EQ(ArrowIpcDecoderDecodeArrayFromShared(
+                &decoder, &shared, 0, &array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
             NANOARROW_OK);
-  EXPECT_EQ(
-      ArrowIpcDecoderValidateArray(&array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
-      NANOARROW_OK);
 
   // Release the original shared (forthcoming array buffers should still be valid)
   ArrowIpcSharedBufferReset(&shared);
@@ -556,11 +559,9 @@ TEST(NanoarrowIpcTest, NanoarrowIpcSharedBufferThreadSafeDecode) {
   struct ArrowArray arrays[10];
   for (int i = 0; i < 10; i++) {
     ASSERT_EQ(
-        ArrowIpcDecoderDecodeArrayFromShared(&decoder, &shared, -1, arrays + i, nullptr),
+        ArrowIpcDecoderDecodeArrayFromShared(&decoder, &shared, -1, arrays + i,
+                                             NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
         NANOARROW_OK);
-    ASSERT_EQ(ArrowIpcDecoderValidateArray(arrays + i, NANOARROW_VALIDATION_LEVEL_FULL,
-                                           nullptr),
-              NANOARROW_OK);
   }
 
   // Clean up
@@ -620,11 +621,9 @@ TEST_P(ArrowTypeParameterizedTestFixture, NanoarrowIpcArrowArrayRoundtrip) {
   ASSERT_EQ(ArrowIpcDecoderDecodeHeader(&decoder, buffer_view, nullptr), NANOARROW_OK);
   buffer_view.data.as_uint8 += decoder.header_size_bytes;
   buffer_view.size_bytes -= decoder.header_size_bytes;
-  ASSERT_EQ(ArrowIpcDecoderDecodeArray(&decoder, buffer_view, -1, &array, nullptr),
+  ASSERT_EQ(ArrowIpcDecoderDecodeArray(&decoder, buffer_view, -1, &array,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
             NANOARROW_OK);
-  ASSERT_EQ(
-      ArrowIpcDecoderValidateArray(&array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
-      NANOARROW_OK);
 
   auto maybe_batch = arrow::ImportRecordBatch(&array, dummy_schema);
   ASSERT_TRUE(maybe_batch.ok());
@@ -640,11 +639,9 @@ TEST_P(ArrowTypeParameterizedTestFixture, NanoarrowIpcArrowArrayRoundtrip) {
   ASSERT_EQ(ArrowIpcDecoderDecodeHeader(&decoder, buffer_view, nullptr), NANOARROW_OK);
   buffer_view.data.as_uint8 += decoder.header_size_bytes;
   buffer_view.size_bytes -= decoder.header_size_bytes;
-  ASSERT_EQ(ArrowIpcDecoderDecodeArray(&decoder, buffer_view, -1, &array, nullptr),
+  ASSERT_EQ(ArrowIpcDecoderDecodeArray(&decoder, buffer_view, -1, &array,
+                                       NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
             NANOARROW_OK);
-  ASSERT_EQ(
-      ArrowIpcDecoderValidateArray(&array, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
-      NANOARROW_OK);
 
   maybe_batch = arrow::ImportRecordBatch(&array, dummy_schema);
   ASSERT_TRUE(maybe_batch.ok());
diff --git a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_reader.c b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_reader.c
index 0e72b3d..15b7c4b 100644
--- a/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_reader.c
+++ b/extensions/nanoarrow_ipc/src/nanoarrow/nanoarrow_ipc_reader.c
@@ -378,23 +378,16 @@ static int ArrowIpcArrayStreamReaderGetNext(struct ArrowArrayStream* stream,
         ArrowIpcSharedBufferInit(&shared, &private_data->body), &private_data->error);
     NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderDecodeArrayFromShared(
         &private_data->decoder, &shared, private_data->field_index, &tmp,
-        &private_data->error));
+        NANOARROW_VALIDATION_LEVEL_FULL, &private_data->error));
     ArrowIpcSharedBufferReset(&shared);
   } else {
     struct ArrowBufferView body_view;
     body_view.data.data = private_data->body.data;
     body_view.size_bytes = private_data->body.size_bytes;
 
-    NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderDecodeArray(&private_data->decoder, body_view,
-                                                       private_data->field_index, &tmp,
-                                                       &private_data->error));
-  }
-
-  result = ArrowIpcDecoderValidateArray(&tmp, NANOARROW_VALIDATION_LEVEL_FULL,
-                                        &private_data->error);
-  if (result != NANOARROW_OK) {
-    tmp.release(&tmp);
-    return result;
+    NANOARROW_RETURN_NOT_OK(ArrowIpcDecoderDecodeArray(
+        &private_data->decoder, body_view, private_data->field_index, &tmp,
+        NANOARROW_VALIDATION_LEVEL_FULL, &private_data->error));
   }
 
   ArrowArrayMove(&tmp, out);
diff --git a/src/nanoarrow/array.c b/src/nanoarrow/array.c
index c669114..fe202b1 100644
--- a/src/nanoarrow/array.c
+++ b/src/nanoarrow/array.c
@@ -165,9 +165,9 @@ ArrowErrorCode ArrowArrayInitFromType(struct ArrowArray* array,
   return NANOARROW_OK;
 }
 
-static ArrowErrorCode ArrowArrayInitFromArrayView(struct ArrowArray* array,
-                                                  struct ArrowArrayView* array_view,
-                                                  struct ArrowError* error) {
+ArrowErrorCode ArrowArrayInitFromArrayView(struct ArrowArray* array,
+                                           struct ArrowArrayView* array_view,
+                                           struct ArrowError* error) {
   ArrowArrayInitFromType(array, array_view->storage_type);
   struct ArrowArrayPrivateData* private_data =
       (struct ArrowArrayPrivateData*)array->private_data;
@@ -1009,15 +1009,16 @@ static int ArrowArrayViewValidateFull(struct ArrowArrayView* array_view,
 
   if (array_view->storage_type == NANOARROW_TYPE_DENSE_UNION ||
       array_view->storage_type == NANOARROW_TYPE_SPARSE_UNION) {
-    // Check that we have valid type ids.
     if (array_view->union_type_id_map == NULL) {
-      // If the union_type_id map is NULL
-      // (e.g., when using ArrowArrayInitFromType() + ArrowArrayAllocateChildren()
-      // + ArrowArrayFinishBuilding()), we don't have enough information to validate
-      // this buffer (GH-178).
-    } else if (_ArrowParsedUnionTypeIdsWillEqualChildIndices(
-                   array_view->union_type_id_map, array_view->n_children,
-                   array_view->n_children)) {
+      // If the union_type_id map is NULL (e.g., when using ArrowArrayInitFromType() +
+      // ArrowArrayAllocateChildren() + ArrowArrayFinishBuilding()), we don't have enough
+      // information to validate this buffer.
+      ArrowErrorSet(error,
+                    "Insufficient information provided for validation of union array");
+      return EINVAL;
+    } else if (_ArrowParsedUnionTypeIdsWillEqualChildIndices(array_view->union_type_id_map,
+                                                      array_view->n_children,
+                                                      array_view->n_children)) {
       NANOARROW_RETURN_NOT_OK(ArrowAssertRangeInt8(
           array_view->buffer_views[0], 0, (int8_t)(array_view->n_children - 1), error));
     } else {
diff --git a/src/nanoarrow/array_test.cc b/src/nanoarrow/array_test.cc
index ea16c57..3170efa 100644
--- a/src/nanoarrow/array_test.cc
+++ b/src/nanoarrow/array_test.cc
@@ -1971,15 +1971,16 @@ TEST(ArrayTest, ArrayViewTestUnionChildIndices) {
   ASSERT_EQ(ArrowArrayFinishUnionElement(&array, 1), NANOARROW_OK);
   ASSERT_EQ(ArrowArrayFinishBuildingDefault(&array, nullptr), NANOARROW_OK);
 
-  // The ArrayView for a union could in theroy be created without a schema,
-  // in which case the type_ids are assumed to equal child indices
+  // The ArrayView for a union could in theroy be created without a schema.
+  // Currently FULL validation will fail here since we can't guarantee that
+  // these are valid.
   ArrowArrayViewInitFromType(&array_view, NANOARROW_TYPE_DENSE_UNION);
   ASSERT_EQ(ArrowArrayViewAllocateChildren(&array_view, 2), NANOARROW_OK);
   ArrowArrayViewInitFromType(array_view.children[0], NANOARROW_TYPE_INT32);
   ArrowArrayViewInitFromType(array_view.children[1], NANOARROW_TYPE_STRING);
   ASSERT_EQ(ArrowArrayViewSetArray(&array_view, &array, nullptr), NANOARROW_OK);
   EXPECT_EQ(ArrowArrayViewValidate(&array_view, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
-            NANOARROW_OK);
+            EINVAL);
 
   EXPECT_EQ(ArrowArrayViewUnionTypeId(&array_view, 0), 0);
   EXPECT_EQ(ArrowArrayViewUnionTypeId(&array_view, 1), 1);
@@ -1988,7 +1989,7 @@ TEST(ArrayTest, ArrayViewTestUnionChildIndices) {
 
   ArrowArrayViewReset(&array_view);
 
-  // The test schema explicitly sets the type_ids 0,1 and this should work too
+  // The test schema explicitly sets the type_ids 0,1 and this should validate properly
   ASSERT_EQ(ArrowArrayViewInitFromSchema(&array_view, &schema, nullptr), NANOARROW_OK);
   ASSERT_EQ(ArrowArrayViewSetArray(&array_view, &array, nullptr), NANOARROW_OK);
   EXPECT_EQ(ArrowArrayViewValidate(&array_view, NANOARROW_VALIDATION_LEVEL_FULL, nullptr),
diff --git a/src/nanoarrow/nanoarrow.h b/src/nanoarrow/nanoarrow.h
index c591b68..41570df 100644
--- a/src/nanoarrow/nanoarrow.h
+++ b/src/nanoarrow/nanoarrow.h
@@ -92,6 +92,8 @@
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowArrayInitFromType)
 #define ArrowArrayInitFromSchema \
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowArrayInitFromSchema)
+#define ArrowArrayInitFromArrayView \
+  NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowArrayInitFromArrayView)
 #define ArrowArrayAllocateChildren \
   NANOARROW_SYMBOL(NANOARROW_NAMESPACE, ArrowArrayAllocateChildren)
 #define ArrowArrayAllocateDictionary \
@@ -752,6 +754,14 @@ ArrowErrorCode ArrowArrayInitFromSchema(struct ArrowArray* array,
                                         struct ArrowSchema* schema,
                                         struct ArrowError* error);
 
+/// \brief Initialize the contents of an ArrowArray from an ArrowArrayView
+///
+/// Caller is responsible for calling the array->release callback if
+/// NANOARROW_OK is returned.
+ArrowErrorCode ArrowArrayInitFromArrayView(struct ArrowArray* array,
+                                           struct ArrowArrayView* array_view,
+                                           struct ArrowError* error);
+
 /// \brief Allocate the array->children array
 ///
 /// Includes the memory for each child struct ArrowArray,