Skip to main content

arrow_avro/reader/
mod.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Avro reader
19//!
20//! Facilities to read Apache Avro–encoded data into Arrow's `RecordBatch` format.
21//!
22//! ### Limitations
23//!
24//!- **Avro unions with > 127 branches are not supported.**
25//!  When decoding Avro unions to Arrow `UnionArray`, Arrow stores the union
26//!  type identifiers in an **8‑bit signed** buffer (`i8`). This implies a
27//!  practical limit of **127** distinct branch ids. Inputs that resolve to
28//!  more than 127 branches will return an error. If you truly need more,
29//!  model the schema as a **union of unions**, per the Arrow format spec.
30//!
31//!  See: Arrow Columnar Format — Dense Union (“types buffer: 8‑bit signed;
32//!  a union with more than 127 possible types can be modeled as a union of
33//!  unions”).
34//!
35//! This module exposes three layers of the API surface, from highest to lowest-level:
36//!
37//! * [`ReaderBuilder`](crate::reader::ReaderBuilder): configures how Avro is read (batch size, strict union handling,
38//!   string representation, reader schema, etc.) and produces either:
39//!   * a `Reader` for **Avro Object Container Files (OCF)** read from any `BufRead`, or
40//!   * a low-level `Decoder` for **single‑object encoded** Avro bytes and Confluent
41//!     **Schema Registry** framed messages.
42//! * [`Reader`](crate::reader::Reader): a convenient, synchronous iterator over `RecordBatch` decoded from an OCF
43//!   input. Implements [`Iterator<Item = Result<RecordBatch, ArrowError>>`] and
44//!   `RecordBatchReader`.
45//! * [`Decoder`](crate::reader::Decoder): a push‑based row decoder that consumes SOE framed Avro bytes and yields ready
46//!   `RecordBatch` values when batches fill. This is suitable for integrating with async
47//!   byte streams, network protocols, or other custom data sources.
48//!
49//! ## Encodings and when to use which type
50//!
51//! * **Object Container File (OCF)**: A self‑describing file format with a header containing
52//!   the writer schema, optional compression codec, and a sync marker, followed by one or
53//!   more data blocks. Use `Reader` for this format. See the Avro 1.11.1 specification
54//!   (“Object Container Files”). <https://avro.apache.org/docs/1.11.1/specification/#object-container-files>
55//! * **Single‑Object Encoding**: A stream‑friendly framing that prefixes each record body with
56//!   the 2‑byte marker `0xC3 0x01` followed by the **8‑byte little‑endian CRC‑64‑AVRO Rabin
57//!   fingerprint** of the writer schema, then the Avro binary body. Use `Decoder` with a
58//!   populated `SchemaStore` to resolve fingerprints to full schemas.
59//!   See “Single object encoding” in the Avro 1.11.1 spec.
60//!   <https://avro.apache.org/docs/1.11.1/specification/#single-object-encoding>
61//! * **Confluent Schema Registry wire format**: A 1‑byte magic `0x00`, a **4‑byte big‑endian**
62//!   schema ID, then the Avro‑encoded body. Use `Decoder` with a `SchemaStore` configured
63//!   for `FingerprintAlgorithm::Id` and entries keyed by `Fingerprint::Id`. See
64//!   Confluent’s “Wire format” documentation.
65//!   <https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/index.html#wire-format>
66//! * **Apicurio Schema Registry wire format**: A 1‑byte magic `0x00`, a **8‑byte big‑endian**
67//!   global schema ID, then the Avro‑encoded body. Use `Decoder` with a `SchemaStore` configured
68//!   for `FingerprintAlgorithm::Id64` and entries keyed by `Fingerprint::Id64`. See
69//!   Apicurio’s “Avro SerDe” documentation.
70//!   <https://www.apicur.io/registry/docs/apicurio-registry/1.3.3.Final/getting-started/assembly-using-kafka-client-serdes.html#registry-serdes-types-avro-registry>
71//!
72//! ## Basic file usage (OCF)
73//!
74//! Use `ReaderBuilder::build` to construct a `Reader` from any `BufRead`. The doctest below
75//! creates a tiny OCF in memory using `AvroWriter` and then reads it back.
76//!
77//! ```
78//! use std::io::Cursor;
79//! use std::sync::Arc;
80//! use arrow_array::{ArrayRef, Int32Array, RecordBatch};
81//! use arrow_schema::{DataType, Field, Schema};
82//! use arrow_avro::writer::AvroWriter;
83//! use arrow_avro::reader::ReaderBuilder;
84//!
85//! # fn main() -> Result<(), Box<dyn std::error::Error>> {
86//! // Build a minimal Arrow schema and batch
87//! let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]);
88//! let batch = RecordBatch::try_new(
89//!     Arc::new(schema.clone()),
90//!     vec![Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef],
91//! )?;
92//!
93//! // Write an Avro OCF to memory
94//! let buffer: Vec<u8> = Vec::new();
95//! let mut writer = AvroWriter::new(buffer, schema.clone())?;
96//! writer.write(&batch)?;
97//! writer.finish()?;
98//! let bytes = writer.into_inner();
99//!
100//! // Read it back with ReaderBuilder
101//! let mut reader = ReaderBuilder::new().build(Cursor::new(bytes))?;
102//! let out = reader.next().unwrap()?;
103//! assert_eq!(out.num_rows(), 3);
104//! # Ok(()) }
105//! ```
106//!
107//! ## Streaming usage (single‑object / Confluent / Apicurio)
108//!
109//! The `Decoder` lets you integrate Avro decoding with **any** source of bytes by
110//! periodically calling `Decoder::decode` with new data and calling `Decoder::flush`
111//! to get a `RecordBatch` once at least one row is complete.
112//!
113//! The example below shows how to decode from an arbitrary stream of `bytes::Bytes` using
114//! `futures` utilities. Note: this is illustrative and keeps a single in‑memory `Bytes`
115//! buffer for simplicity—real applications typically maintain a rolling buffer.
116//!
117//! ```
118//! use bytes::{Buf, Bytes};
119//! use futures::{Stream, StreamExt};
120//! use std::task::{Poll, ready};
121//! use arrow_array::RecordBatch;
122//! use arrow_avro::{reader::Decoder, errors::AvroError};
123//!
124//! /// Decode a stream of Avro-framed bytes into RecordBatch values.
125//! fn decode_stream<S: Stream<Item = Bytes> + Unpin>(
126//!     mut decoder: Decoder,
127//!     mut input: S,
128//! ) -> impl Stream<Item = Result<RecordBatch, AvroError>> {
129//!     let mut buffered = Bytes::new();
130//!     futures::stream::poll_fn(move |cx| {
131//!         loop {
132//!             if buffered.is_empty() {
133//!                 buffered = match ready!(input.poll_next_unpin(cx)) {
134//!                     Some(b) => b,
135//!                     None => break, // EOF
136//!                 };
137//!             }
138//!             // Feed as much as possible
139//!             let decoded = match decoder.decode(buffered.as_ref()) {
140//!                 Ok(n) => n,
141//!                 Err(e) => return Poll::Ready(Some(Err(e))),
142//!             };
143//!             let read = buffered.len();
144//!             buffered.advance(decoded);
145//!             if decoded != read {
146//!                 // decoder made partial progress; request more bytes
147//!                 break
148//!             }
149//!         }
150//!         // Return a batch if one or more rows are complete
151//!         Poll::Ready(decoder.flush().transpose())
152//!     })
153//! }
154//! ```
155//!
156//! ### Building and using a `Decoder` for **single‑object encoding** (Rabin fingerprints)
157//!
158//! The doctest below **writes** a single‑object framed record using the Avro writer
159//! (no manual varints) for the writer schema
160//! (`{"type":"record","name":"User","fields":[{"name":"id","type":"long"}]}`)
161//! and then decodes it into a `RecordBatch`.
162//!
163//! ```
164//! use std::sync::Arc;
165//! use std::collections::HashMap;
166//! use arrow_array::{ArrayRef, Int64Array, RecordBatch};
167//! use arrow_schema::{DataType, Field, Schema};
168//! use arrow_avro::schema::{AvroSchema, SchemaStore, SCHEMA_METADATA_KEY, FingerprintStrategy};
169//! use arrow_avro::writer::{WriterBuilder, format::AvroSoeFormat};
170//! use arrow_avro::reader::ReaderBuilder;
171//!
172//! # fn main() -> Result<(), Box<dyn std::error::Error>> {
173//! // Register the writer schema (Rabin fingerprint by default).
174//! let mut store = SchemaStore::new();
175//! let avro_schema = AvroSchema::new(r#"{"type":"record","name":"User","fields":[
176//!   {"name":"id","type":"long"}]}"#.to_string());
177//! let _fp = store.register(avro_schema.clone())?;
178//!
179//! // Create a single-object framed record { id: 42 } with the Avro writer.
180//! let mut md = HashMap::new();
181//! md.insert(SCHEMA_METADATA_KEY.to_string(), avro_schema.json_string.clone());
182//! let arrow = Schema::new_with_metadata(vec![Field::new("id", DataType::Int64, false)], md);
183//! let batch = RecordBatch::try_new(
184//!     Arc::new(arrow.clone()),
185//!     vec![Arc::new(Int64Array::from(vec![42])) as ArrayRef],
186//! )?;
187//! let mut w = WriterBuilder::new(arrow)
188//!     .with_fingerprint_strategy(FingerprintStrategy::Rabin) // SOE prefix
189//!     .build::<_, AvroSoeFormat>(Vec::new())?;
190//! w.write(&batch)?;
191//! w.finish()?;
192//! let frame = w.into_inner(); // C3 01 + fp + Avro body
193//!
194//! // Decode with a `Decoder`
195//! let mut dec = ReaderBuilder::new()
196//!   .with_writer_schema_store(store)
197//!   .with_batch_size(1024)
198//!   .build_decoder()?;
199//!
200//! dec.decode(&frame)?;
201//! let out = dec.flush()?.expect("one batch");
202//! assert_eq!(out.num_rows(), 1);
203//! # Ok(()) }
204//! ```
205//!
206//! See Avro 1.11.1 “Single object encoding” for details of the 2‑byte marker
207//! and little‑endian CRC‑64‑AVRO fingerprint:
208//! <https://avro.apache.org/docs/1.11.1/specification/#single-object-encoding>
209//!
210//! ### Building and using a `Decoder` for **Confluent Schema Registry** framing
211//!
212//! The Confluent wire format is: 1‑byte magic `0x00`, then a **4‑byte big‑endian** schema ID,
213//! then the Avro body. The doctest below crafts two messages for the same schema ID and
214//! decodes them into a single `RecordBatch` with two rows.
215//!
216//! ```
217//! use std::sync::Arc;
218//! use std::collections::HashMap;
219//! use arrow_array::{ArrayRef, Int64Array, StringArray, RecordBatch};
220//! use arrow_schema::{DataType, Field, Schema};
221//! use arrow_avro::schema::{AvroSchema, SchemaStore, Fingerprint, FingerprintAlgorithm, SCHEMA_METADATA_KEY, FingerprintStrategy};
222//! use arrow_avro::writer::{WriterBuilder, format::AvroSoeFormat};
223//! use arrow_avro::reader::ReaderBuilder;
224//!
225//! # fn main() -> Result<(), Box<dyn std::error::Error>> {
226//! // Set up a store keyed by numeric IDs (Confluent).
227//! let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
228//! let schema_id = 7u32;
229//! let avro_schema = AvroSchema::new(r#"{"type":"record","name":"User","fields":[
230//!   {"name":"id","type":"long"}, {"name":"name","type":"string"}]}"#.to_string());
231//! store.set(Fingerprint::Id(schema_id), avro_schema.clone())?;
232//!
233//! // Write two Confluent-framed messages {id:1,name:"a"} and {id:2,name:"b"}.
234//! fn msg(id: i64, name: &str, schema: &AvroSchema, schema_id: u32) -> Result<Vec<u8>, Box<dyn std::error::Error>> {
235//!     let mut md = HashMap::new();
236//!     md.insert(SCHEMA_METADATA_KEY.to_string(), schema.json_string.clone());
237//!     let arrow = Schema::new_with_metadata(
238//!         vec![Field::new("id", DataType::Int64, false), Field::new("name", DataType::Utf8, false)],
239//!         md,
240//!     );
241//!     let batch = RecordBatch::try_new(
242//!         Arc::new(arrow.clone()),
243//!         vec![
244//!           Arc::new(Int64Array::from(vec![id])) as ArrayRef,
245//!           Arc::new(StringArray::from(vec![name])) as ArrayRef,
246//!         ],
247//!     )?;
248//!     let mut w = WriterBuilder::new(arrow)
249//!         .with_fingerprint_strategy(FingerprintStrategy::Id(schema_id)) // 0x00 + ID + body
250//!         .build::<_, AvroSoeFormat>(Vec::new())?;
251//!     w.write(&batch)?; w.finish()?;
252//!     Ok(w.into_inner())
253//! }
254//! let m1 = msg(1, "a", &avro_schema, schema_id)?;
255//! let m2 = msg(2, "b", &avro_schema, schema_id)?;
256//!
257//! // Decode both into a single batch.
258//! let mut dec = ReaderBuilder::new()
259//!   .with_writer_schema_store(store)
260//!   .with_batch_size(1024)
261//!   .build_decoder()?;
262//! dec.decode(&m1)?;
263//! dec.decode(&m2)?;
264//! let batch = dec.flush()?.expect("batch");
265//! assert_eq!(batch.num_rows(), 2);
266//! # Ok(()) }
267//! ```
268//!
269//! See Confluent’s “Wire format” notes: magic byte `0x00`, 4‑byte **big‑endian** schema ID,
270//! then the Avro‑encoded payload.
271//! <https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/index.html#wire-format>
272//!
273//! ## Schema resolution (reader vs. writer schemas)
274//!
275//! Avro supports resolving data written with one schema (“writer”) into another (“reader”)
276//! using rules like **field aliases**, **default values**, and **numeric promotions**.
277//! In practice this lets you evolve schemas over time while remaining compatible with old data.
278//!
279//! *Spec background:* See Avro’s **Schema Resolution** (aliases, defaults) and the Confluent
280//! **Wire format** (magic `0x00` + big‑endian schema id + Avro body).
281//! <https://avro.apache.org/docs/1.11.1/specification/#schema-resolution>
282//! <https://docs.confluent.io/platform/current/schema-registry/fundamentals/serdes-develop/index.html#wire-format>
283//!
284//! ### OCF example: rename a field and add a default via a reader schema
285//!
286//! Below we write an OCF with a *writer schema* having fields `id: long`, `name: string`.
287//! We then read it with a *reader schema* that:
288//! - **renames** `name` to `full_name` via `aliases`, and
289//! - **adds** `is_active: boolean` with a **default** value `true`.
290//!
291//! ```
292//! use std::io::Cursor;
293//! use std::sync::Arc;
294//! use arrow_array::{ArrayRef, Int64Array, StringArray, RecordBatch};
295//! use arrow_schema::{DataType, Field, Schema};
296//! use arrow_avro::writer::AvroWriter;
297//! use arrow_avro::reader::ReaderBuilder;
298//! use arrow_avro::schema::AvroSchema;
299//!
300//! # fn main() -> Result<(), Box<dyn std::error::Error>> {
301//! // Writer (past version): { id: long, name: string }
302//! let writer_arrow = Schema::new(vec![
303//!     Field::new("id", DataType::Int64, false),
304//!     Field::new("name", DataType::Utf8, false),
305//! ]);
306//! let batch = RecordBatch::try_new(
307//!     Arc::new(writer_arrow.clone()),
308//!     vec![
309//!         Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef,
310//!         Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
311//!     ],
312//! )?;
313//!
314//! // Write an OCF entirely in memory
315//! let mut w = AvroWriter::new(Vec::<u8>::new(), writer_arrow)?;
316//! w.write(&batch)?;
317//! w.finish()?;
318//! let bytes = w.into_inner();
319//!
320//! // Reader (current version):
321//! //  - record name "topLevelRecord" matches the crate's default for OCF
322//! //  - rename `name` -> `full_name` using aliases (optional)
323//! let reader_json = r#"
324//! {
325//!   "type": "record",
326//!   "name": "topLevelRecord",
327//!   "fields": [
328//!     { "name": "id", "type": "long" },
329//!     { "name": "full_name", "type": ["null","string"], "aliases": ["name"], "default": null },
330//!     { "name": "is_active", "type": "boolean", "default": true }
331//!   ]
332//! }"#;
333//!
334//! let mut reader = ReaderBuilder::new()
335//!   .with_reader_schema(AvroSchema::new(reader_json.to_string()))
336//!   .build(Cursor::new(bytes))?;
337//!
338//! let out = reader.next().unwrap()?;
339//! assert_eq!(out.num_rows(), 2);
340//! # Ok(()) }
341//! ```
342//!
343//! ### Confluent single‑object example: resolve *past* writer versions to the topic’s **current** reader schema
344//!
345//! In this scenario, the **reader schema** is the topic’s *current* schema, while the two
346//! **writer schemas** registered under Confluent IDs **1** and **2** represent *past versions*.
347//! The decoder uses the reader schema to resolve both versions.
348//!
349//! ```
350//! use std::sync::Arc;
351//! use std::collections::HashMap;
352//! use arrow_avro::reader::ReaderBuilder;
353//! use arrow_avro::schema::{
354//!     AvroSchema, Fingerprint, FingerprintAlgorithm, SchemaStore,
355//!     SCHEMA_METADATA_KEY, FingerprintStrategy,
356//! };
357//! use arrow_array::{ArrayRef, Int32Array, Int64Array, StringArray, RecordBatch};
358//! use arrow_schema::{DataType, Field, Schema};
359//!
360//! fn main() -> Result<(), Box<dyn std::error::Error>> {
361//!     // Reader: current topic schema (no reader-added fields)
362//!     //   {"type":"record","name":"User","fields":[
363//!     //     {"name":"id","type":"long"},
364//!     //     {"name":"name","type":"string"}]}
365//!     let reader_schema = AvroSchema::new(
366//!         r#"{"type":"record","name":"User",
367//!             "fields":[{"name":"id","type":"long"},{"name":"name","type":"string"}]}"#
368//!             .to_string(),
369//!     );
370//!
371//!     // Register two *writer* schemas under Confluent IDs 0 and 1
372//!     let writer_v0 = AvroSchema::new(
373//!         r#"{"type":"record","name":"User",
374//!             "fields":[{"name":"id","type":"int"},{"name":"name","type":"string"}]}"#
375//!             .to_string(),
376//!     );
377//!     let writer_v1 = AvroSchema::new(
378//!         r#"{"type":"record","name":"User",
379//!             "fields":[{"name":"id","type":"long"},{"name":"name","type":"string"},
380//!                       {"name":"email","type":["null","string"],"default":null}]}"#
381//!             .to_string(),
382//!     );
383//!
384//!     let id_v0: u32 = 0;
385//!     let id_v1: u32 = 1;
386//!
387//!     let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id); // integer IDs
388//!     store.set(Fingerprint::Id(id_v0), writer_v0.clone())?;
389//!     store.set(Fingerprint::Id(id_v1), writer_v1.clone())?;
390//!
391//!     // Write two Confluent-framed messages using each writer version
392//!     // frame0: writer v0 body {id:1001_i32, name:"v0-alice"}
393//!     let mut md0 = HashMap::new();
394//!     md0.insert(SCHEMA_METADATA_KEY.to_string(), writer_v0.json_string.clone());
395//!     let arrow0 = Schema::new_with_metadata(
396//!         vec![Field::new("id", DataType::Int32, false),
397//!              Field::new("name", DataType::Utf8, false)], md0);
398//!     let batch0 = RecordBatch::try_new(
399//!         Arc::new(arrow0.clone()),
400//!         vec![Arc::new(Int32Array::from(vec![1001])) as ArrayRef,
401//!              Arc::new(StringArray::from(vec!["v0-alice"])) as ArrayRef])?;
402//!     let mut w0 = arrow_avro::writer::WriterBuilder::new(arrow0)
403//!         .with_fingerprint_strategy(FingerprintStrategy::Id(id_v0))
404//!         .build::<_, arrow_avro::writer::format::AvroSoeFormat>(Vec::new())?;
405//!     w0.write(&batch0)?; w0.finish()?;
406//!     let frame0 = w0.into_inner(); // 0x00 + id_v0 + body
407//!
408//!     // frame1: writer v1 body {id:2002_i64, name:"v1-bob", email: Some("bob@example.com")}
409//!     let mut md1 = HashMap::new();
410//!    md1.insert(SCHEMA_METADATA_KEY.to_string(), writer_v1.json_string.clone());
411//!     let arrow1 = Schema::new_with_metadata(
412//!         vec![Field::new("id", DataType::Int64, false),
413//!              Field::new("name", DataType::Utf8, false),
414//!              Field::new("email", DataType::Utf8, true)], md1);
415//!     let batch1 = RecordBatch::try_new(
416//!         Arc::new(arrow1.clone()),
417//!         vec![Arc::new(Int64Array::from(vec![2002])) as ArrayRef,
418//!              Arc::new(StringArray::from(vec!["v1-bob"])) as ArrayRef,
419//!              Arc::new(StringArray::from(vec![Some("bob@example.com")])) as ArrayRef])?;
420//!     let mut w1 = arrow_avro::writer::WriterBuilder::new(arrow1)
421//!         .with_fingerprint_strategy(FingerprintStrategy::Id(id_v1))
422//!         .build::<_, arrow_avro::writer::format::AvroSoeFormat>(Vec::new())?;
423//!     w1.write(&batch1)?; w1.finish()?;
424//!     let frame1 = w1.into_inner(); // 0x00 + id_v1 + body
425//!
426//!     // Build a streaming Decoder that understands Confluent framing
427//!     let mut decoder = ReaderBuilder::new()
428//!         .with_reader_schema(reader_schema)
429//!         .with_writer_schema_store(store)
430//!         .with_batch_size(8) // small demo batches
431//!         .build_decoder()?;
432//!
433//!     // Decode each whole frame, then drain completed rows with flush()
434//!     let mut total_rows = 0usize;
435//!
436//!     let consumed0 = decoder.decode(&frame0)?;
437//!     assert_eq!(consumed0, frame0.len(), "decoder must consume the whole frame");
438//!     while let Some(batch) = decoder.flush()? { total_rows += batch.num_rows(); }
439//!
440//!     let consumed1 = decoder.decode(&frame1)?;
441//!     assert_eq!(consumed1, frame1.len(), "decoder must consume the whole frame");
442//!     while let Some(batch) = decoder.flush()? { total_rows += batch.num_rows(); }
443//!
444//!     // We sent 2 records so we should get 2 rows (possibly one per flush)
445//!     assert_eq!(total_rows, 2);
446//!     Ok(())
447//! }
448//! ```
449//!
450//! ## Schema evolution and batch boundaries
451//!
452//! `Decoder` supports mid‑stream schema changes when the input framing carries a schema
453//! fingerprint (single‑object or Confluent). When a new fingerprint is observed:
454//!
455//! * If the current `RecordBatch` is **empty**, the decoder switches to the new schema
456//!   immediately.
457//! * If not, the decoder finishes the current batch first and only then switches.
458//!
459//! Consequently, the schema of batches produced by `Decoder::flush` may change over time,
460//! and `Decoder` intentionally does **not** implement `RecordBatchReader`. In contrast,
461//! `Reader` (OCF) has a single writer schema for the entire file and therefore implements
462//! `RecordBatchReader`.
463//!
464//! ## Performance & memory
465//!
466//! * `batch_size` controls the maximum number of rows per `RecordBatch`. Larger batches
467//!   amortize per‑batch overhead; smaller batches reduce peak memory usage and latency.
468//! * When `utf8_view` is enabled, string columns use Arrow’s `StringViewArray`, which can
469//!   reduce allocations for short strings.
470//! * For OCF, blocks may be compressed; `Reader` will decompress using the codec specified
471//!   in the file header and feed uncompressed bytes to the row `Decoder`.
472//!
473//! ## Error handling
474//!
475//! * Incomplete inputs return parse errors with "Unexpected EOF"; callers typically provide
476//!   more bytes and try again.
477//! * If a fingerprint is unknown to the provided `SchemaStore`, decoding fails with a
478//!   descriptive error. Populate the store up front to avoid this.
479//!
480//! ---
481use crate::codec::AvroFieldBuilder;
482use crate::errors::AvroError;
483use crate::reader::header::read_header;
484use crate::schema::{
485    AvroSchema, CONFLUENT_MAGIC, Fingerprint, FingerprintAlgorithm, SCHEMA_METADATA_KEY,
486    SINGLE_OBJECT_MAGIC, Schema, SchemaStore,
487};
488use arrow_array::{RecordBatch, RecordBatchReader};
489use arrow_schema::{ArrowError, SchemaRef};
490use block::BlockDecoder;
491use header::Header;
492use indexmap::IndexMap;
493use record::RecordDecoder;
494use std::io::BufRead;
495
496mod block;
497mod cursor;
498mod header;
499mod record;
500mod vlq;
501
502#[cfg(feature = "async")]
503mod async_reader;
504
505#[cfg(feature = "object_store")]
506pub use async_reader::AvroObjectReader;
507#[cfg(feature = "async")]
508pub use async_reader::{AsyncAvroFileReader, AsyncFileReader};
509
510fn is_incomplete_data(err: &AvroError) -> bool {
511    matches!(
512        err,
513        AvroError::EOF(_) | AvroError::NeedMoreData(_) | AvroError::NeedMoreDataRange(_)
514    )
515}
516
517/// A low‑level, push‑based decoder from Avro bytes to Arrow `RecordBatch`.
518///
519/// `Decoder` is designed for **streaming** scenarios:
520///
521/// * You *feed* freshly received bytes using `Self::decode`, potentially multiple times,
522///   until at least one row is complete.
523/// * You then *drain* completed rows with `Self::flush`, which yields a `RecordBatch`
524///   if any rows were finished since the last flush.
525///
526/// Unlike `Reader`, which is specialized for Avro **Object Container Files**, `Decoder`
527/// understands **framed single‑object** inputs and **Confluent Schema Registry** messages,
528/// switching schemas mid‑stream when the framing indicates a new fingerprint.
529///
530/// ### Supported prefixes
531///
532/// On each new row boundary, `Decoder` tries to match one of the following "prefixes":
533///
534/// * **Single‑Object encoding**: magic `0xC3 0x01` + schema fingerprint (length depends on
535///   the configured `FingerprintAlgorithm`); see `SINGLE_OBJECT_MAGIC`.
536/// * **Confluent wire format**: magic `0x00` + 4‑byte big‑endian schema id; see
537///   `CONFLUENT_MAGIC`.
538///
539/// The active fingerprint determines which cached row decoder is used to decode the following
540/// record body bytes.
541///
542/// ### Schema switching semantics
543///
544/// When a new fingerprint is observed:
545///
546/// * If the current batch is empty, the decoder switches immediately;
547/// * Otherwise, the current batch is finalized on the next `flush` and only then
548///   does the decoder switch to the new schema. This guarantees that a single `RecordBatch`
549///   never mixes rows with different schemas.
550///
551/// ### Examples
552///
553/// Build and use a `Decoder` for single‑object encoding:
554///
555/// ```
556/// use arrow_avro::schema::{AvroSchema, SchemaStore};
557/// use arrow_avro::reader::ReaderBuilder;
558///
559/// # fn main() -> Result<(), Box<dyn std::error::Error>> {
560/// // Use a record schema at the top level so we can build an Arrow RecordBatch
561/// let mut store = SchemaStore::new(); // Rabin fingerprinting by default
562/// let avro = AvroSchema::new(
563///     r#"{"type":"record","name":"E","fields":[{"name":"x","type":"long"}]}"#.to_string()
564/// );
565/// let fp = store.register(avro)?;
566///
567/// // --- Hidden: write a single-object framed row {x:7} ---
568/// # use std::sync::Arc;
569/// # use std::collections::HashMap;
570/// # use arrow_array::{ArrayRef, Int64Array, RecordBatch};
571/// # use arrow_schema::{DataType, Field, Schema};
572/// # use arrow_avro::schema::{SCHEMA_METADATA_KEY, FingerprintStrategy};
573/// # use arrow_avro::writer::{WriterBuilder, format::AvroSoeFormat};
574/// # let mut md = HashMap::new();
575/// # md.insert(SCHEMA_METADATA_KEY.to_string(),
576/// #     r#"{"type":"record","name":"E","fields":[{"name":"x","type":"long"}]}"#.to_string());
577/// # let arrow = Schema::new_with_metadata(vec![Field::new("x", DataType::Int64, false)], md);
578/// # let batch = RecordBatch::try_new(Arc::new(arrow.clone()), vec![Arc::new(Int64Array::from(vec![7])) as ArrayRef])?;
579/// # let mut w = WriterBuilder::new(arrow)
580/// #     .with_fingerprint_strategy(fp.into())
581/// #     .build::<_, AvroSoeFormat>(Vec::new())?;
582/// # w.write(&batch)?; w.finish()?; let frame = w.into_inner();
583///
584/// let mut decoder = ReaderBuilder::new()
585///     .with_writer_schema_store(store)
586///     .with_batch_size(16)
587///     .build_decoder()?;
588///
589/// # decoder.decode(&frame)?;
590/// let batch = decoder.flush()?.expect("one row");
591/// assert_eq!(batch.num_rows(), 1);
592/// # Ok(()) }
593/// ```
594///
595/// *Background:* Avro's single‑object encoding is defined as `0xC3 0x01` + 8‑byte
596/// little‑endian CRC‑64‑AVRO fingerprint of the **writer schema** + Avro binary body.
597/// See the Avro 1.11.1 spec for details. <https://avro.apache.org/docs/1.11.1/specification/#single-object-encoding>
598///
599/// Build and use a `Decoder` for Confluent Registry messages:
600///
601/// ```
602/// use arrow_avro::schema::{AvroSchema, SchemaStore, Fingerprint, FingerprintAlgorithm};
603/// use arrow_avro::reader::ReaderBuilder;
604///
605/// # fn main() -> Result<(), Box<dyn std::error::Error>> {
606/// let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
607/// store.set(Fingerprint::Id(1234), AvroSchema::new(r#"{"type":"record","name":"E","fields":[{"name":"x","type":"long"}]}"#.to_string()))?;
608///
609/// // --- Hidden: encode two Confluent-framed messages {x:1} and {x:2} ---
610/// # use std::sync::Arc;
611/// # use std::collections::HashMap;
612/// # use arrow_array::{ArrayRef, Int64Array, RecordBatch};
613/// # use arrow_schema::{DataType, Field, Schema};
614/// # use arrow_avro::schema::{SCHEMA_METADATA_KEY, FingerprintStrategy};
615/// # use arrow_avro::writer::{WriterBuilder, format::AvroSoeFormat};
616/// # fn msg(x: i64) -> Result<Vec<u8>, Box<dyn std::error::Error>> {
617/// #   let mut md = HashMap::new();
618/// #   md.insert(SCHEMA_METADATA_KEY.to_string(),
619/// #     r#"{"type":"record","name":"E","fields":[{"name":"x","type":"long"}]}"#.to_string());
620/// #   let arrow = Schema::new_with_metadata(vec![Field::new("x", DataType::Int64, false)], md);
621/// #   let batch = RecordBatch::try_new(Arc::new(arrow.clone()), vec![Arc::new(Int64Array::from(vec![x])) as ArrayRef])?;
622/// #   let mut w = WriterBuilder::new(arrow)
623/// #       .with_fingerprint_strategy(FingerprintStrategy::Id(1234))
624/// #       .build::<_, AvroSoeFormat>(Vec::new())?;
625/// #   w.write(&batch)?; w.finish()?; Ok(w.into_inner())
626/// # }
627/// # let m1 = msg(1)?;
628/// # let m2 = msg(2)?;
629///
630/// let mut decoder = ReaderBuilder::new()
631///     .with_writer_schema_store(store)
632///     .build_decoder()?;
633/// # decoder.decode(&m1)?;
634/// # decoder.decode(&m2)?;
635/// let batch = decoder.flush()?.expect("two rows");
636/// assert_eq!(batch.num_rows(), 2);
637/// # Ok(()) }
638/// ```
639#[derive(Debug)]
640pub struct Decoder {
641    active_decoder: RecordDecoder,
642    active_fingerprint: Option<Fingerprint>,
643    batch_size: usize,
644    remaining_capacity: usize,
645    cache: IndexMap<Fingerprint, RecordDecoder>,
646    fingerprint_algorithm: FingerprintAlgorithm,
647    pending_schema: Option<(Fingerprint, RecordDecoder)>,
648    awaiting_body: bool,
649}
650
651impl Decoder {
652    pub(crate) fn from_parts(
653        batch_size: usize,
654        active_decoder: RecordDecoder,
655        active_fingerprint: Option<Fingerprint>,
656        cache: IndexMap<Fingerprint, RecordDecoder>,
657        fingerprint_algorithm: FingerprintAlgorithm,
658    ) -> Self {
659        Self {
660            batch_size,
661            remaining_capacity: batch_size,
662            active_fingerprint,
663            active_decoder,
664            cache,
665            fingerprint_algorithm,
666            pending_schema: None,
667            awaiting_body: false,
668        }
669    }
670
671    /// Returns the Arrow schema for the rows decoded by this decoder.
672    ///
673    /// **Note:** With single‑object or Confluent framing, the schema may change
674    /// at a row boundary when the input indicates a new fingerprint.
675    pub fn schema(&self) -> SchemaRef {
676        self.active_decoder.schema().clone()
677    }
678
679    /// Returns the configured maximum number of rows per batch.
680    pub fn batch_size(&self) -> usize {
681        self.batch_size
682    }
683
684    /// Feed a chunk of bytes into the decoder.
685    ///
686    /// This will:
687    ///
688    /// * Decode at most `Self::batch_size` rows;
689    /// * Return the number of input bytes **consumed** from `data` (which may be 0 if more
690    ///   bytes are required, or less than `data.len()` if a prefix/body straddles the
691    ///   chunk boundary);
692    /// * Defer producing a `RecordBatch` until you call `Self::flush`.
693    ///
694    /// # Returns
695    /// The number of bytes consumed from `data`.
696    ///
697    /// # Errors
698    /// Returns an error if:
699    ///
700    /// * The input indicates an unknown fingerprint (not present in the provided
701    ///   `SchemaStore`;
702    /// * The Avro body is malformed;
703    /// * A strict‑mode union rule is violated (see `ReaderBuilder::with_strict_mode`).
704    pub fn decode(&mut self, data: &[u8]) -> Result<usize, AvroError> {
705        let mut total_consumed = 0usize;
706        while total_consumed < data.len() && self.remaining_capacity > 0 {
707            if self.awaiting_body {
708                match self.active_decoder.decode(&data[total_consumed..], 1) {
709                    Ok(n) => {
710                        self.remaining_capacity -= 1;
711                        total_consumed += n;
712                        self.awaiting_body = false;
713                        continue;
714                    }
715                    Err(ref e) if is_incomplete_data(e) => break,
716                    Err(e) => return Err(e),
717                };
718            }
719            match self.handle_prefix(&data[total_consumed..])? {
720                Some(0) => break, // Insufficient bytes
721                Some(n) => {
722                    total_consumed += n;
723                    self.apply_pending_schema_if_batch_empty();
724                    self.awaiting_body = true;
725                }
726                None => {
727                    return Err(AvroError::ParseError(
728                        "Missing magic bytes and fingerprint".to_string(),
729                    ));
730                }
731            }
732        }
733        Ok(total_consumed)
734    }
735
736    // Attempt to handle a prefix at the current position.
737    // * Ok(None) – buffer does not start with the prefix.
738    // * Ok(Some(0)) – prefix detected, but the buffer is too short; caller should await more bytes.
739    // * Ok(Some(n)) – consumed `n > 0` bytes of a complete prefix (magic and fingerprint).
740    fn handle_prefix(&mut self, buf: &[u8]) -> Result<Option<usize>, AvroError> {
741        match self.fingerprint_algorithm {
742            FingerprintAlgorithm::Rabin => {
743                self.handle_prefix_common(buf, &SINGLE_OBJECT_MAGIC, |bytes| {
744                    Fingerprint::Rabin(u64::from_le_bytes(bytes))
745                })
746            }
747            FingerprintAlgorithm::Id => self.handle_prefix_common(buf, &CONFLUENT_MAGIC, |bytes| {
748                Fingerprint::Id(u32::from_be_bytes(bytes))
749            }),
750            FingerprintAlgorithm::Id64 => {
751                self.handle_prefix_common(buf, &CONFLUENT_MAGIC, |bytes| {
752                    Fingerprint::Id64(u64::from_be_bytes(bytes))
753                })
754            }
755            #[cfg(feature = "md5")]
756            FingerprintAlgorithm::MD5 => {
757                self.handle_prefix_common(buf, &SINGLE_OBJECT_MAGIC, |bytes| {
758                    Fingerprint::MD5(bytes)
759                })
760            }
761            #[cfg(feature = "sha256")]
762            FingerprintAlgorithm::SHA256 => {
763                self.handle_prefix_common(buf, &SINGLE_OBJECT_MAGIC, |bytes| {
764                    Fingerprint::SHA256(bytes)
765                })
766            }
767        }
768    }
769
770    /// This method checks for the provided `magic` bytes at the start of `buf` and, if present,
771    /// attempts to read the following fingerprint of `N` bytes, converting it to a
772    /// `Fingerprint` using `fingerprint_from`.
773    fn handle_prefix_common<const MAGIC_LEN: usize, const N: usize>(
774        &mut self,
775        buf: &[u8],
776        magic: &[u8; MAGIC_LEN],
777        fingerprint_from: impl FnOnce([u8; N]) -> Fingerprint,
778    ) -> Result<Option<usize>, AvroError> {
779        // Need at least the magic bytes to decide
780        // 2 bytes for Avro Spec and 1 byte for Confluent Wire Protocol.
781        if buf.len() < MAGIC_LEN {
782            return Ok(Some(0));
783        }
784        // Bail out early if the magic does not match.
785        if &buf[..MAGIC_LEN] != magic {
786            return Ok(None);
787        }
788        // Try to parse the fingerprint that follows the magic.
789        let consumed_fp = self.handle_fingerprint(&buf[MAGIC_LEN..], fingerprint_from)?;
790        // Convert the inner result into a “bytes consumed” count.
791        // NOTE: Incomplete fingerprint consumes no bytes.
792        Ok(Some(consumed_fp.map_or(0, |n| n + MAGIC_LEN)))
793    }
794
795    // Attempts to read and install a new fingerprint of `N` bytes.
796    //
797    // * Ok(None) – insufficient bytes (`buf.len() < `N`).
798    // * Ok(Some(N)) – fingerprint consumed (always `N`).
799    fn handle_fingerprint<const N: usize>(
800        &mut self,
801        buf: &[u8],
802        fingerprint_from: impl FnOnce([u8; N]) -> Fingerprint,
803    ) -> Result<Option<usize>, AvroError> {
804        // Need enough bytes to get fingerprint (next N bytes)
805        let Some(fingerprint_bytes) = buf.get(..N) else {
806            return Ok(None); // insufficient bytes
807        };
808        // SAFETY: length checked above.
809        let new_fingerprint = fingerprint_from(fingerprint_bytes.try_into().unwrap());
810        // If the fingerprint indicates a schema change, prepare to switch decoders.
811        if self.active_fingerprint != Some(new_fingerprint) {
812            let Some(new_decoder) = self.cache.shift_remove(&new_fingerprint) else {
813                return Err(AvroError::ParseError(format!(
814                    "Unknown fingerprint: {new_fingerprint:?}"
815                )));
816            };
817            self.pending_schema = Some((new_fingerprint, new_decoder));
818            // If there are already decoded rows, we must flush them first.
819            // Reducing `remaining_capacity` to 0 ensures `flush` is called next.
820            if self.remaining_capacity < self.batch_size {
821                self.remaining_capacity = 0;
822            }
823        }
824        Ok(Some(N))
825    }
826
827    fn apply_pending_schema(&mut self) {
828        if let Some((new_fingerprint, new_decoder)) = self.pending_schema.take() {
829            if let Some(old_fingerprint) = self.active_fingerprint.replace(new_fingerprint) {
830                let old_decoder = std::mem::replace(&mut self.active_decoder, new_decoder);
831                self.cache.shift_remove(&old_fingerprint);
832                self.cache.insert(old_fingerprint, old_decoder);
833            } else {
834                self.active_decoder = new_decoder;
835            }
836        }
837    }
838
839    fn apply_pending_schema_if_batch_empty(&mut self) {
840        if self.batch_is_empty() {
841            self.apply_pending_schema();
842        }
843    }
844
845    fn flush_and_reset(&mut self) -> Result<Option<RecordBatch>, AvroError> {
846        if self.batch_is_empty() {
847            return Ok(None);
848        }
849        let batch = self.active_decoder.flush()?;
850        self.remaining_capacity = self.batch_size;
851        Ok(Some(batch))
852    }
853
854    /// Produce a `RecordBatch` if at least one row is fully decoded, returning
855    /// `Ok(None)` if no new rows are available.
856    ///
857    /// If a schema change was detected while decoding rows for the current batch, the
858    /// schema switch is applied **after** flushing this batch, so the **next** batch
859    /// (if any) may have a different schema.
860    pub fn flush(&mut self) -> Result<Option<RecordBatch>, AvroError> {
861        // We must flush the active decoder before switching to the pending one.
862        let batch = self.flush_and_reset();
863        self.apply_pending_schema();
864        batch
865    }
866
867    /// Returns the number of rows that can be added to this decoder before it is full.
868    pub fn capacity(&self) -> usize {
869        self.remaining_capacity
870    }
871
872    /// Returns true if the decoder has reached its capacity for the current batch.
873    pub fn batch_is_full(&self) -> bool {
874        self.remaining_capacity == 0
875    }
876
877    /// Returns true if the decoder has not decoded any batches yet (i.e., the current batch is empty).
878    pub fn batch_is_empty(&self) -> bool {
879        self.remaining_capacity == self.batch_size
880    }
881
882    // Decode either the block count or remaining capacity from `data` (an OCF block payload).
883    //
884    // Returns the number of bytes consumed from `data` along with the number of records decoded.
885    fn decode_block(&mut self, data: &[u8], count: usize) -> Result<(usize, usize), AvroError> {
886        // OCF decoding never interleaves records across blocks, so no chunking.
887        let to_decode = std::cmp::min(count, self.remaining_capacity);
888        if to_decode == 0 {
889            return Ok((0, 0));
890        }
891        let consumed = self.active_decoder.decode(data, to_decode)?;
892        self.remaining_capacity -= to_decode;
893        Ok((consumed, to_decode))
894    }
895
896    // Produce a `RecordBatch` if at least one row is fully decoded, returning
897    // `Ok(None)` if no new rows are available.
898    fn flush_block(&mut self) -> Result<Option<RecordBatch>, AvroError> {
899        self.flush_and_reset()
900    }
901}
902
903/// A builder that configures and constructs Avro readers and decoders.
904///
905/// `ReaderBuilder` is the primary entry point for this module. It supports:
906///
907/// * OCF reading via `Self::build`, returning a `Reader` over any `BufRead`;
908/// * streaming decoding via `Self::build_decoder`, returning a `Decoder`.
909///
910/// ### Options
911///
912/// * **`batch_size`**: Max rows per `RecordBatch` (default: `1024`). See `Self::with_batch_size`.
913/// * **`utf8_view`**: Use Arrow `StringViewArray` for string columns (default: `false`).
914///   See `Self::with_utf8_view`.
915/// * **`strict_mode`**: Opt‑in to stricter union handling (default: `false`).
916///   See `Self::with_strict_mode`.
917/// * **`reader_schema`**: Optional reader schema (projection / evolution) used when decoding
918///   values (default: `None`). See `Self::with_reader_schema`.
919/// * **`projection`**: Optional projection of **top‑level record fields** by index (default: `None`).
920///
921///   If set, the effective reader schema is **pruned** to include only the projected fields, in the
922///   specified order:
923///
924///   * If a reader schema is provided, that schema is pruned.
925///   * Otherwise, a reader schema is derived from the writer schema and then pruned.
926///   * For streaming `Decoder` with multiple writer schemas and no reader schema, a projected reader
927///     schema is derived **per writer schema** in the `SchemaStore`.
928///
929///   See `Self::with_projection`.
930/// * **`writer_schema_store`**: Required for building a `Decoder` for single‑object or
931///   Confluent framing. Maps fingerprints to Avro schemas. See `Self::with_writer_schema_store`.
932/// * **`active_fingerprint`**: Optional starting fingerprint for streaming decode when the
933///   first frame omits one (rare). See `Self::with_active_fingerprint`.
934///
935/// ### Examples
936///
937/// Read an OCF file in batches of 4096 rows:
938///
939/// ```no_run
940/// use std::fs::File;
941/// use std::io::BufReader;
942/// use arrow_avro::reader::ReaderBuilder;
943///
944/// let file = File::open("data.avro")?;
945/// let mut reader = ReaderBuilder::new()
946///     .with_batch_size(4096)
947///     .build(BufReader::new(file))?;
948/// # Ok::<(), Box<dyn std::error::Error>>(())
949/// ```
950///
951/// Build a `Decoder` for Confluent messages:
952///
953/// ```
954/// use arrow_avro::schema::{AvroSchema, SchemaStore, Fingerprint, FingerprintAlgorithm};
955/// use arrow_avro::reader::ReaderBuilder;
956///
957/// let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
958/// store.set(Fingerprint::Id(1234), AvroSchema::new(r#"{"type":"record","name":"E","fields":[]}"#.to_string()))?;
959///
960/// let decoder = ReaderBuilder::new()
961///     .with_writer_schema_store(store)
962///     .build_decoder()?;
963/// # Ok::<(), Box<dyn std::error::Error>>(())
964/// ```
965#[derive(Debug)]
966pub struct ReaderBuilder {
967    batch_size: usize,
968    strict_mode: bool,
969    utf8_view: bool,
970    reader_schema: Option<AvroSchema>,
971    projection: Option<Vec<usize>>,
972    writer_schema_store: Option<SchemaStore>,
973    active_fingerprint: Option<Fingerprint>,
974}
975
976impl Default for ReaderBuilder {
977    fn default() -> Self {
978        Self {
979            batch_size: 1024,
980            strict_mode: false,
981            utf8_view: false,
982            reader_schema: None,
983            projection: None,
984            writer_schema_store: None,
985            active_fingerprint: None,
986        }
987    }
988}
989
990impl ReaderBuilder {
991    /// Creates a new `ReaderBuilder` with defaults:
992    ///
993    /// * `batch_size = 1024`
994    /// * `strict_mode = false`
995    /// * `utf8_view = false`
996    /// * `reader_schema = None`
997    /// * `projection = None`
998    /// * `writer_schema_store = None`
999    /// * `active_fingerprint = None`
1000    pub fn new() -> Self {
1001        Self::default()
1002    }
1003
1004    fn make_record_decoder(
1005        &self,
1006        writer_schema: &Schema,
1007        reader_schema: Option<&Schema>,
1008    ) -> Result<RecordDecoder, AvroError> {
1009        let mut builder = AvroFieldBuilder::new(writer_schema);
1010        if let Some(reader_schema) = reader_schema {
1011            builder = builder.with_reader_schema(reader_schema);
1012        }
1013        let root = builder
1014            .with_utf8view(self.utf8_view)
1015            .with_strict_mode(self.strict_mode)
1016            .build()?;
1017        RecordDecoder::try_new_with_options(root.data_type())
1018    }
1019
1020    fn make_record_decoder_from_schemas(
1021        &self,
1022        writer_schema: &Schema,
1023        reader_schema: Option<&AvroSchema>,
1024    ) -> Result<RecordDecoder, AvroError> {
1025        let reader_schema_raw = reader_schema.map(|s| s.schema()).transpose()?;
1026        self.make_record_decoder(writer_schema, reader_schema_raw.as_ref())
1027    }
1028
1029    fn make_decoder(
1030        &self,
1031        header: Option<&Header>,
1032        reader_schema: Option<&AvroSchema>,
1033    ) -> Result<Decoder, AvroError> {
1034        if let Some(hdr) = header {
1035            let writer_schema = hdr.schema()?.ok_or_else(|| {
1036                AvroError::ParseError("No Avro schema present in file header".into())
1037            })?;
1038            let projected_reader_schema = self
1039                .projection
1040                .as_deref()
1041                .map(|projection| {
1042                    let base_schema = if let Some(reader_schema) = reader_schema {
1043                        reader_schema.clone()
1044                    } else {
1045                        let raw = hdr.get(SCHEMA_METADATA_KEY).ok_or_else(|| {
1046                            AvroError::ParseError(
1047                                "No Avro schema present in file header".to_string(),
1048                            )
1049                        })?;
1050                        let json_string = std::str::from_utf8(raw)
1051                            .map_err(|e| {
1052                                AvroError::ParseError(format!(
1053                                    "Invalid UTF-8 in Avro schema header: {e}"
1054                                ))
1055                            })?
1056                            .to_string();
1057                        AvroSchema::new(json_string)
1058                    };
1059                    base_schema.project(projection)
1060                })
1061                .transpose()?;
1062            let effective_reader_schema = projected_reader_schema.as_ref().or(reader_schema);
1063            let record_decoder =
1064                self.make_record_decoder_from_schemas(&writer_schema, effective_reader_schema)?;
1065            return Ok(Decoder::from_parts(
1066                self.batch_size,
1067                record_decoder,
1068                None,
1069                IndexMap::new(),
1070                FingerprintAlgorithm::Rabin,
1071            ));
1072        }
1073        let store = self.writer_schema_store.as_ref().ok_or_else(|| {
1074            AvroError::ParseError("Writer schema store required for raw Avro".into())
1075        })?;
1076        let fingerprints = store.fingerprints();
1077        if fingerprints.is_empty() {
1078            return Err(AvroError::ParseError(
1079                "Writer schema store must contain at least one schema".into(),
1080            ));
1081        }
1082        let start_fingerprint = self
1083            .active_fingerprint
1084            .or_else(|| fingerprints.first().copied())
1085            .ok_or_else(|| {
1086                AvroError::ParseError("Could not determine initial schema fingerprint".into())
1087            })?;
1088        let projection = self.projection.as_deref();
1089        let projected_reader_schema = match (projection, reader_schema) {
1090            (Some(projection), Some(reader_schema)) => Some(reader_schema.project(projection)?),
1091            _ => None,
1092        };
1093        let mut cache = IndexMap::with_capacity(fingerprints.len().saturating_sub(1));
1094        let mut active_decoder: Option<RecordDecoder> = None;
1095        for fingerprint in store.fingerprints() {
1096            let avro_schema = match store.lookup(&fingerprint) {
1097                Some(schema) => schema,
1098                None => {
1099                    return Err(AvroError::General(format!(
1100                        "Fingerprint {fingerprint:?} not found in schema store",
1101                    )));
1102                }
1103            };
1104            let writer_schema = avro_schema.schema()?;
1105            let record_decoder = match projection {
1106                None => self.make_record_decoder_from_schemas(&writer_schema, reader_schema)?,
1107                Some(projection) => {
1108                    if let Some(ref pruned_reader_schema) = projected_reader_schema {
1109                        self.make_record_decoder_from_schemas(
1110                            &writer_schema,
1111                            Some(pruned_reader_schema),
1112                        )?
1113                    } else {
1114                        let derived_reader_schema = avro_schema.project(projection)?;
1115                        self.make_record_decoder_from_schemas(
1116                            &writer_schema,
1117                            Some(&derived_reader_schema),
1118                        )?
1119                    }
1120                }
1121            };
1122            if fingerprint == start_fingerprint {
1123                active_decoder = Some(record_decoder);
1124            } else {
1125                cache.insert(fingerprint, record_decoder);
1126            }
1127        }
1128        let active_decoder = active_decoder.ok_or_else(|| {
1129            AvroError::General(format!(
1130                "Initial fingerprint {start_fingerprint:?} not found in schema store"
1131            ))
1132        })?;
1133        Ok(Decoder::from_parts(
1134            self.batch_size,
1135            active_decoder,
1136            Some(start_fingerprint),
1137            cache,
1138            store.fingerprint_algorithm(),
1139        ))
1140    }
1141
1142    /// Sets the **row‑based batch size**.
1143    ///
1144    /// Each call to `Decoder::flush` or each iteration of `Reader` yields a batch with
1145    /// *up to* this many rows. Larger batches can reduce overhead; smaller batches can
1146    /// reduce peak memory usage and latency.
1147    pub fn with_batch_size(mut self, batch_size: usize) -> Self {
1148        self.batch_size = batch_size;
1149        self
1150    }
1151
1152    /// Choose Arrow's `StringViewArray` for UTF‑8 string data.
1153    ///
1154    /// When enabled, textual Avro fields are loaded into Arrow’s **StringViewArray**
1155    /// instead of the standard `StringArray`. This can improve performance for workloads
1156    /// with many short strings by reducing allocations.
1157    pub fn with_utf8_view(mut self, utf8_view: bool) -> Self {
1158        self.utf8_view = utf8_view;
1159        self
1160    }
1161
1162    /// Returns whether `StringViewArray` is enabled for string data.
1163    pub fn use_utf8view(&self) -> bool {
1164        self.utf8_view
1165    }
1166
1167    /// Enable stricter behavior for certain Avro unions (e.g., `[T, "null"]`).
1168    ///
1169    /// When `true`, ambiguous or lossy unions that would otherwise be coerced may instead
1170    /// produce a descriptive error. Use this to catch schema issues early during ingestion.
1171    pub fn with_strict_mode(mut self, strict_mode: bool) -> Self {
1172        self.strict_mode = strict_mode;
1173        self
1174    }
1175
1176    /// Sets the **reader schema** used during decoding.
1177    ///
1178    /// If not provided, the writer schema from the OCF header (for `Reader`) or the
1179    /// schema looked up from the fingerprint (for `Decoder`) is used directly.
1180    ///
1181    /// A reader schema can be used for **schema evolution** or **projection**.
1182    pub fn with_reader_schema(mut self, schema: AvroSchema) -> Self {
1183        self.reader_schema = Some(schema);
1184        self
1185    }
1186
1187    /// Sets an explicit top-level field projection by index.
1188    ///
1189    /// The provided `projection` is a list of indices into the **top-level record** fields.
1190    /// The output schema will contain only these fields, in the specified order.
1191    ///
1192    /// Internally, this is implemented by pruning the effective Avro *reader schema*:
1193    ///
1194    /// * If a reader schema is provided via `Self::with_reader_schema`, that schema is pruned.
1195    /// * Otherwise, a reader schema is derived from the writer schema and then pruned.
1196    /// * For streaming `Decoder` with multiple writer schemas and no reader schema, a projected
1197    ///   reader schema is derived **per writer schema** in the `SchemaStore`.
1198    ///
1199    /// # Example
1200    ///
1201    /// Read only specific columns from an Avro OCF file:
1202    ///
1203    /// ```
1204    /// use std::io::Cursor;
1205    /// use std::sync::Arc;
1206    /// use arrow_array::{ArrayRef, Int32Array, StringArray, Float64Array, RecordBatch};
1207    /// use arrow_schema::{DataType, Field, Schema};
1208    /// use arrow_avro::writer::AvroWriter;
1209    /// use arrow_avro::reader::ReaderBuilder;
1210    ///
1211    /// # fn main() -> Result<(), Box<dyn std::error::Error>> {
1212    /// // Original schema has three fields: id, name, value
1213    /// let schema = Schema::new(vec![
1214    ///     Field::new("id", DataType::Int32, false),
1215    ///     Field::new("name", DataType::Utf8, false),
1216    ///     Field::new("value", DataType::Float64, false),
1217    /// ]);
1218    /// let batch = RecordBatch::try_new(
1219    ///     Arc::new(schema.clone()),
1220    ///     vec![
1221    ///         Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
1222    ///         Arc::new(StringArray::from(vec!["a", "b", "c"])) as ArrayRef,
1223    ///         Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0])) as ArrayRef,
1224    ///     ],
1225    /// )?;
1226    ///
1227    /// // Write Avro OCF
1228    /// let mut writer = AvroWriter::new(Vec::new(), schema)?;
1229    /// writer.write(&batch)?;
1230    /// writer.finish()?;
1231    /// let bytes = writer.into_inner();
1232    ///
1233    /// // Read only fields at indices 2 and 0 (value, id) — in that order
1234    /// let mut reader = ReaderBuilder::new()
1235    ///     .with_projection(vec![2, 0])
1236    ///     .build(Cursor::new(bytes))?;
1237    ///
1238    /// let out = reader.next().unwrap()?;
1239    /// assert_eq!(out.num_columns(), 2);
1240    /// assert_eq!(out.schema().field(0).name(), "value");
1241    /// assert_eq!(out.schema().field(1).name(), "id");
1242    /// # Ok(()) }
1243    /// ```
1244    pub fn with_projection(mut self, projection: Vec<usize>) -> Self {
1245        self.projection = Some(projection);
1246        self
1247    }
1248
1249    /// Sets the `SchemaStore` used to resolve writer schemas by fingerprint.
1250    ///
1251    /// This is required when building a `Decoder` for **single‑object encoding** or the
1252    /// **Confluent** wire format. The store maps a fingerprint (Rabin / MD5 / SHA‑256 /
1253    /// ID) to a full Avro schema.
1254    ///
1255    /// Defaults to `None`.
1256    pub fn with_writer_schema_store(mut self, store: SchemaStore) -> Self {
1257        self.writer_schema_store = Some(store);
1258        self
1259    }
1260
1261    /// Sets the initial schema fingerprint for stream decoding.
1262    ///
1263    /// This can be useful for streams that **do not include** a fingerprint before the first
1264    /// record body (uncommon). If not set, the first observed fingerprint is used.
1265    pub fn with_active_fingerprint(mut self, fp: Fingerprint) -> Self {
1266        self.active_fingerprint = Some(fp);
1267        self
1268    }
1269
1270    /// Build a `Reader` (OCF) from this builder and a `BufRead`.
1271    ///
1272    /// This reads and validates the OCF header, initializes an internal row decoder from
1273    /// the discovered writer (and optional reader) schema, and prepares to iterate blocks,
1274    /// decompressing if necessary.
1275    pub fn build<R: BufRead>(self, mut reader: R) -> Result<Reader<R>, ArrowError> {
1276        let header = read_header(&mut reader)?;
1277        let decoder = self.make_decoder(Some(&header), self.reader_schema.as_ref())?;
1278        Ok(Reader {
1279            reader,
1280            header,
1281            decoder,
1282            block_decoder: BlockDecoder::default(),
1283            block_data: Vec::new(),
1284            block_count: 0,
1285            block_cursor: 0,
1286            finished: false,
1287        })
1288    }
1289
1290    /// Build a streaming `Decoder` from this builder.
1291    ///
1292    /// # Requirements
1293    /// * `SchemaStore` **must** be provided via `Self::with_writer_schema_store`.
1294    /// * The store should contain **all** fingerprints that may appear on the stream.
1295    ///
1296    /// # Errors
1297    /// * Returns [`ArrowError::InvalidArgumentError`] if the schema store is missing
1298    pub fn build_decoder(self) -> Result<Decoder, ArrowError> {
1299        if self.writer_schema_store.is_none() {
1300            return Err(ArrowError::InvalidArgumentError(
1301                "Building a decoder requires a writer schema store".to_string(),
1302            ));
1303        }
1304        self.make_decoder(None, self.reader_schema.as_ref())
1305            .map_err(ArrowError::from)
1306    }
1307}
1308
1309/// A high‑level Avro **Object Container File** reader.
1310///
1311/// `Reader` pulls blocks from a `BufRead` source, handles optional block compression,
1312/// and decodes them row‑by‑row into Arrow `RecordBatch` values using an internal
1313/// `Decoder`. It implements both:
1314///
1315/// * [`Iterator<Item = Result<RecordBatch, ArrowError>>`], and
1316/// * `RecordBatchReader`, guaranteeing a consistent schema across all produced batches.
1317///
1318#[derive(Debug)]
1319pub struct Reader<R: BufRead> {
1320    reader: R,
1321    header: Header,
1322    decoder: Decoder,
1323    block_decoder: BlockDecoder,
1324    block_data: Vec<u8>,
1325    block_count: usize,
1326    block_cursor: usize,
1327    finished: bool,
1328}
1329
1330impl<R: BufRead> Reader<R> {
1331    /// Returns the Arrow schema discovered from the Avro file header (or derived via
1332    /// the optional reader schema).
1333    pub fn schema(&self) -> SchemaRef {
1334        self.decoder.schema()
1335    }
1336
1337    /// Returns a reference to the parsed Avro container‑file header (magic, metadata, codec, sync).
1338    pub fn avro_header(&self) -> &Header {
1339        &self.header
1340    }
1341
1342    /// Reads the next `RecordBatch` from the Avro file, or `Ok(None)` on EOF.
1343    ///
1344    /// Batches are bounded by `batch_size`; a single OCF block may yield multiple batches,
1345    /// and a batch may also span multiple blocks.
1346    fn read(&mut self) -> Result<Option<RecordBatch>, AvroError> {
1347        'outer: while !self.finished && !self.decoder.batch_is_full() {
1348            while self.block_cursor == self.block_data.len() {
1349                let buf = self.reader.fill_buf()?;
1350                if buf.is_empty() {
1351                    self.finished = true;
1352                    break 'outer;
1353                }
1354                // Try to decode another block from the buffered reader.
1355                let consumed = self.block_decoder.decode(buf)?;
1356                self.reader.consume(consumed);
1357                if let Some(block) = self.block_decoder.flush() {
1358                    // Successfully decoded a block.
1359                    self.block_data = if let Some(ref codec) = self.header.compression()? {
1360                        let decompressed: Vec<u8> = codec.decompress(&block.data)?;
1361                        decompressed
1362                    } else {
1363                        block.data
1364                    };
1365                    self.block_count = block.count;
1366                    self.block_cursor = 0;
1367                } else if consumed == 0 {
1368                    // The block decoder made no progress on a non-empty buffer.
1369                    return Err(AvroError::ParseError(
1370                        "Could not decode next Avro block from partial data".to_string(),
1371                    ));
1372                }
1373            }
1374            // Decode as many rows as will fit in the current batch
1375            if self.block_cursor < self.block_data.len() {
1376                let (consumed, records_decoded) = self
1377                    .decoder
1378                    .decode_block(&self.block_data[self.block_cursor..], self.block_count)?;
1379                self.block_cursor += consumed;
1380                self.block_count -= records_decoded;
1381            }
1382        }
1383        self.decoder.flush_block()
1384    }
1385}
1386
1387impl<R: BufRead> Iterator for Reader<R> {
1388    type Item = Result<RecordBatch, ArrowError>;
1389
1390    fn next(&mut self) -> Option<Self::Item> {
1391        self.read().map_err(ArrowError::from).transpose()
1392    }
1393}
1394
1395impl<R: BufRead> RecordBatchReader for Reader<R> {
1396    fn schema(&self) -> SchemaRef {
1397        self.schema()
1398    }
1399}
1400
1401#[cfg(test)]
1402mod test {
1403    use crate::codec::AvroFieldBuilder;
1404    use crate::reader::record::RecordDecoder;
1405    use crate::reader::{Decoder, Reader, ReaderBuilder};
1406    use crate::schema::{
1407        AVRO_ENUM_SYMBOLS_METADATA_KEY, AVRO_NAME_METADATA_KEY, AVRO_NAMESPACE_METADATA_KEY,
1408        AvroSchema, CONFLUENT_MAGIC, Fingerprint, FingerprintAlgorithm, PrimitiveType,
1409        SINGLE_OBJECT_MAGIC, SchemaStore,
1410    };
1411    use crate::test_util::arrow_test_data;
1412    use crate::writer::AvroWriter;
1413    use arrow_array::builder::{
1414        ArrayBuilder, BooleanBuilder, Float32Builder, Int32Builder, Int64Builder, ListBuilder,
1415        MapBuilder, StringBuilder, StructBuilder,
1416    };
1417    #[cfg(feature = "snappy")]
1418    use arrow_array::builder::{Float64Builder, MapFieldNames};
1419    use arrow_array::cast::AsArray;
1420    #[cfg(not(feature = "avro_custom_types"))]
1421    use arrow_array::types::Int64Type;
1422    #[cfg(feature = "avro_custom_types")]
1423    use arrow_array::types::{
1424        DurationMicrosecondType, DurationMillisecondType, DurationNanosecondType,
1425        DurationSecondType,
1426    };
1427    use arrow_array::types::{Int32Type, IntervalMonthDayNanoType};
1428    use arrow_array::*;
1429    #[cfg(feature = "snappy")]
1430    use arrow_buffer::{Buffer, NullBuffer};
1431    use arrow_buffer::{IntervalMonthDayNano, OffsetBuffer, ScalarBuffer, i256};
1432    #[cfg(feature = "avro_custom_types")]
1433    use arrow_schema::{
1434        ArrowError, DataType, Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit, UnionFields,
1435        UnionMode,
1436    };
1437    #[cfg(not(feature = "avro_custom_types"))]
1438    use arrow_schema::{
1439        ArrowError, DataType, Field, FieldRef, Fields, IntervalUnit, Schema, UnionFields, UnionMode,
1440    };
1441    use bytes::Bytes;
1442    use futures::executor::block_on;
1443    use futures::{Stream, StreamExt, TryStreamExt, stream};
1444    use serde_json::{Value, json};
1445    use std::collections::HashMap;
1446    use std::fs::File;
1447    use std::io::{BufReader, Cursor};
1448    use std::sync::Arc;
1449
1450    fn files() -> impl Iterator<Item = &'static str> {
1451        [
1452            // TODO: avoid requiring snappy for this file
1453            #[cfg(feature = "snappy")]
1454            "avro/alltypes_plain.avro",
1455            #[cfg(feature = "snappy")]
1456            "avro/alltypes_plain.snappy.avro",
1457            #[cfg(feature = "zstd")]
1458            "avro/alltypes_plain.zstandard.avro",
1459            #[cfg(feature = "bzip2")]
1460            "avro/alltypes_plain.bzip2.avro",
1461            #[cfg(feature = "xz")]
1462            "avro/alltypes_plain.xz.avro",
1463        ]
1464        .into_iter()
1465    }
1466
1467    fn read_file(path: &str, batch_size: usize, utf8_view: bool) -> RecordBatch {
1468        let file = File::open(path).unwrap();
1469        let reader = ReaderBuilder::new()
1470            .with_batch_size(batch_size)
1471            .with_utf8_view(utf8_view)
1472            .build(BufReader::new(file))
1473            .unwrap();
1474        let schema = reader.schema();
1475        let batches = reader.collect::<Result<Vec<_>, _>>().unwrap();
1476        arrow::compute::concat_batches(&schema, &batches).unwrap()
1477    }
1478
1479    fn read_file_strict(
1480        path: &str,
1481        batch_size: usize,
1482        utf8_view: bool,
1483    ) -> Result<Reader<BufReader<File>>, ArrowError> {
1484        let file = File::open(path)?;
1485        ReaderBuilder::new()
1486            .with_batch_size(batch_size)
1487            .with_utf8_view(utf8_view)
1488            .with_strict_mode(true)
1489            .build(BufReader::new(file))
1490    }
1491
1492    fn decode_stream<S: Stream<Item = Bytes> + Unpin>(
1493        mut decoder: Decoder,
1494        mut input: S,
1495    ) -> impl Stream<Item = Result<RecordBatch, ArrowError>> {
1496        async_stream::try_stream! {
1497            if let Some(data) = input.next().await {
1498                let consumed = decoder.decode(&data)?;
1499                if consumed < data.len() {
1500                    Err(ArrowError::ParseError(
1501                        "did not consume all bytes".to_string(),
1502                    ))?;
1503                }
1504            }
1505            if let Some(batch) = decoder.flush()? {
1506                yield batch
1507            }
1508        }
1509    }
1510
1511    fn make_record_schema(pt: PrimitiveType) -> AvroSchema {
1512        let js = format!(
1513            r#"{{"type":"record","name":"TestRecord","fields":[{{"name":"a","type":"{}"}}]}}"#,
1514            pt.as_ref()
1515        );
1516        AvroSchema::new(js)
1517    }
1518
1519    fn make_two_schema_store() -> (
1520        SchemaStore,
1521        Fingerprint,
1522        Fingerprint,
1523        AvroSchema,
1524        AvroSchema,
1525    ) {
1526        let schema_int = make_record_schema(PrimitiveType::Int);
1527        let schema_long = make_record_schema(PrimitiveType::Long);
1528        let mut store = SchemaStore::new();
1529        let fp_int = store
1530            .register(schema_int.clone())
1531            .expect("register int schema");
1532        let fp_long = store
1533            .register(schema_long.clone())
1534            .expect("register long schema");
1535        (store, fp_int, fp_long, schema_int, schema_long)
1536    }
1537
1538    fn make_prefix(fp: Fingerprint) -> Vec<u8> {
1539        match fp {
1540            Fingerprint::Rabin(v) => {
1541                let mut out = Vec::with_capacity(2 + 8);
1542                out.extend_from_slice(&SINGLE_OBJECT_MAGIC);
1543                out.extend_from_slice(&v.to_le_bytes());
1544                out
1545            }
1546            Fingerprint::Id(v) => {
1547                panic!("make_prefix expects a Rabin fingerprint, got ({v})");
1548            }
1549            Fingerprint::Id64(v) => {
1550                panic!("make_prefix expects a Rabin fingerprint, got ({v})");
1551            }
1552            #[cfg(feature = "md5")]
1553            Fingerprint::MD5(v) => {
1554                panic!("make_prefix expects a Rabin fingerprint, got ({v:?})");
1555            }
1556            #[cfg(feature = "sha256")]
1557            Fingerprint::SHA256(id) => {
1558                panic!("make_prefix expects a Rabin fingerprint, got ({id:?})");
1559            }
1560        }
1561    }
1562
1563    fn make_decoder(store: &SchemaStore, fp: Fingerprint, reader_schema: &AvroSchema) -> Decoder {
1564        ReaderBuilder::new()
1565            .with_batch_size(8)
1566            .with_reader_schema(reader_schema.clone())
1567            .with_writer_schema_store(store.clone())
1568            .with_active_fingerprint(fp)
1569            .build_decoder()
1570            .expect("decoder")
1571    }
1572
1573    fn make_id_prefix(id: u32, additional: usize) -> Vec<u8> {
1574        let capacity = CONFLUENT_MAGIC.len() + size_of::<u32>() + additional;
1575        let mut out = Vec::with_capacity(capacity);
1576        out.extend_from_slice(&CONFLUENT_MAGIC);
1577        out.extend_from_slice(&id.to_be_bytes());
1578        out
1579    }
1580
1581    fn make_message_id(id: u32, value: i64) -> Vec<u8> {
1582        let encoded_value = encode_zigzag(value);
1583        let mut msg = make_id_prefix(id, encoded_value.len());
1584        msg.extend_from_slice(&encoded_value);
1585        msg
1586    }
1587
1588    fn make_id64_prefix(id: u64, additional: usize) -> Vec<u8> {
1589        let capacity = CONFLUENT_MAGIC.len() + size_of::<u64>() + additional;
1590        let mut out = Vec::with_capacity(capacity);
1591        out.extend_from_slice(&CONFLUENT_MAGIC);
1592        out.extend_from_slice(&id.to_be_bytes());
1593        out
1594    }
1595
1596    fn make_message_id64(id: u64, value: i64) -> Vec<u8> {
1597        let encoded_value = encode_zigzag(value);
1598        let mut msg = make_id64_prefix(id, encoded_value.len());
1599        msg.extend_from_slice(&encoded_value);
1600        msg
1601    }
1602
1603    fn make_value_schema(pt: PrimitiveType) -> AvroSchema {
1604        let json_schema = format!(
1605            r#"{{"type":"record","name":"S","fields":[{{"name":"v","type":"{}"}}]}}"#,
1606            pt.as_ref()
1607        );
1608        AvroSchema::new(json_schema)
1609    }
1610
1611    fn encode_zigzag(value: i64) -> Vec<u8> {
1612        let mut n = ((value << 1) ^ (value >> 63)) as u64;
1613        let mut out = Vec::new();
1614        loop {
1615            if (n & !0x7F) == 0 {
1616                out.push(n as u8);
1617                break;
1618            } else {
1619                out.push(((n & 0x7F) | 0x80) as u8);
1620                n >>= 7;
1621            }
1622        }
1623        out
1624    }
1625
1626    fn make_message(fp: Fingerprint, value: i64) -> Vec<u8> {
1627        let mut msg = make_prefix(fp);
1628        msg.extend_from_slice(&encode_zigzag(value));
1629        msg
1630    }
1631
1632    fn load_writer_schema_json(path: &str) -> Value {
1633        let file = File::open(path).unwrap();
1634        let header = super::read_header(BufReader::new(file)).unwrap();
1635        let schema = header.schema().unwrap().unwrap();
1636        serde_json::to_value(&schema).unwrap()
1637    }
1638
1639    fn make_reader_schema_with_promotions(
1640        path: &str,
1641        promotions: &HashMap<&str, &str>,
1642    ) -> AvroSchema {
1643        let mut root = load_writer_schema_json(path);
1644        assert_eq!(root["type"], "record", "writer schema must be a record");
1645        let fields = root
1646            .get_mut("fields")
1647            .and_then(|f| f.as_array_mut())
1648            .expect("record has fields");
1649        for f in fields.iter_mut() {
1650            let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
1651                continue;
1652            };
1653            if let Some(new_ty) = promotions.get(name) {
1654                let ty = f.get_mut("type").expect("field has a type");
1655                match ty {
1656                    Value::String(_) => {
1657                        *ty = Value::String((*new_ty).to_string());
1658                    }
1659                    // Union
1660                    Value::Array(arr) => {
1661                        for b in arr.iter_mut() {
1662                            match b {
1663                                Value::String(s) if s != "null" => {
1664                                    *b = Value::String((*new_ty).to_string());
1665                                    break;
1666                                }
1667                                Value::Object(_) => {
1668                                    *b = Value::String((*new_ty).to_string());
1669                                    break;
1670                                }
1671                                _ => {}
1672                            }
1673                        }
1674                    }
1675                    Value::Object(_) => {
1676                        *ty = Value::String((*new_ty).to_string());
1677                    }
1678                    _ => {}
1679                }
1680            }
1681        }
1682        AvroSchema::new(root.to_string())
1683    }
1684
1685    fn make_reader_schema_with_enum_remap(
1686        path: &str,
1687        remap: &HashMap<&str, Vec<&str>>,
1688    ) -> AvroSchema {
1689        let mut root = load_writer_schema_json(path);
1690        assert_eq!(root["type"], "record", "writer schema must be a record");
1691        let fields = root
1692            .get_mut("fields")
1693            .and_then(|f| f.as_array_mut())
1694            .expect("record has fields");
1695
1696        fn to_symbols_array(symbols: &[&str]) -> Value {
1697            Value::Array(symbols.iter().map(|s| Value::String((*s).into())).collect())
1698        }
1699
1700        fn update_enum_symbols(ty: &mut Value, symbols: &Value) {
1701            match ty {
1702                Value::Object(map) => {
1703                    if matches!(map.get("type"), Some(Value::String(t)) if t == "enum") {
1704                        map.insert("symbols".to_string(), symbols.clone());
1705                    }
1706                }
1707                Value::Array(arr) => {
1708                    for b in arr.iter_mut() {
1709                        if let Value::Object(map) = b {
1710                            if matches!(map.get("type"), Some(Value::String(t)) if t == "enum") {
1711                                map.insert("symbols".to_string(), symbols.clone());
1712                            }
1713                        }
1714                    }
1715                }
1716                _ => {}
1717            }
1718        }
1719        for f in fields.iter_mut() {
1720            let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
1721                continue;
1722            };
1723            if let Some(new_symbols) = remap.get(name) {
1724                let symbols_val = to_symbols_array(new_symbols);
1725                let ty = f.get_mut("type").expect("field has a type");
1726                update_enum_symbols(ty, &symbols_val);
1727            }
1728        }
1729        AvroSchema::new(root.to_string())
1730    }
1731
1732    fn read_alltypes_with_reader_schema(path: &str, reader_schema: AvroSchema) -> RecordBatch {
1733        let file = File::open(path).unwrap();
1734        let reader = ReaderBuilder::new()
1735            .with_batch_size(1024)
1736            .with_utf8_view(false)
1737            .with_reader_schema(reader_schema)
1738            .build(BufReader::new(file))
1739            .unwrap();
1740        let schema = reader.schema();
1741        let batches = reader.collect::<Result<Vec<_>, _>>().unwrap();
1742        arrow::compute::concat_batches(&schema, &batches).unwrap()
1743    }
1744
1745    fn make_reader_schema_with_selected_fields_in_order(
1746        path: &str,
1747        selected: &[&str],
1748    ) -> AvroSchema {
1749        let mut root = load_writer_schema_json(path);
1750        assert_eq!(root["type"], "record", "writer schema must be a record");
1751        let writer_fields = root
1752            .get("fields")
1753            .and_then(|f| f.as_array())
1754            .expect("record has fields");
1755        let mut field_map: HashMap<String, Value> = HashMap::with_capacity(writer_fields.len());
1756        for f in writer_fields {
1757            if let Some(name) = f.get("name").and_then(|n| n.as_str()) {
1758                field_map.insert(name.to_string(), f.clone());
1759            }
1760        }
1761        let mut new_fields = Vec::with_capacity(selected.len());
1762        for name in selected {
1763            let f = field_map
1764                .get(*name)
1765                .unwrap_or_else(|| panic!("field '{name}' not found in writer schema"))
1766                .clone();
1767            new_fields.push(f);
1768        }
1769        root["fields"] = Value::Array(new_fields);
1770        AvroSchema::new(root.to_string())
1771    }
1772
1773    fn write_ocf(schema: &Schema, batches: &[RecordBatch]) -> Vec<u8> {
1774        let mut w = AvroWriter::new(Vec::<u8>::new(), schema.clone()).expect("writer");
1775        for b in batches {
1776            w.write(b).expect("write");
1777        }
1778        w.finish().expect("finish");
1779        w.into_inner()
1780    }
1781
1782    #[test]
1783    fn ocf_projection_no_reader_schema_reorder() -> Result<(), Box<dyn std::error::Error>> {
1784        // Writer: { id: int, name: string, is_active: boolean }
1785        let writer_schema = Schema::new(vec![
1786            Field::new("id", DataType::Int32, false),
1787            Field::new("name", DataType::Utf8, false),
1788            Field::new("is_active", DataType::Boolean, false),
1789        ]);
1790        let batch = RecordBatch::try_new(
1791            Arc::new(writer_schema.clone()),
1792            vec![
1793                Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef,
1794                Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
1795                Arc::new(BooleanArray::from(vec![true, false])) as ArrayRef,
1796            ],
1797        )?;
1798        let bytes = write_ocf(&writer_schema, &[batch]);
1799        // Project and reorder: [is_active, id]
1800        let mut reader = ReaderBuilder::new()
1801            .with_projection(vec![2, 0])
1802            .build(Cursor::new(bytes))?;
1803        let out = reader.next().unwrap()?;
1804        assert_eq!(out.num_columns(), 2);
1805        assert_eq!(out.schema().field(0).name(), "is_active");
1806        assert_eq!(out.schema().field(1).name(), "id");
1807        let is_active = out.column(0).as_boolean();
1808        assert!(is_active.value(0));
1809        assert!(!is_active.value(1));
1810        let id = out.column(1).as_primitive::<Int32Type>();
1811        assert_eq!(id.value(0), 1);
1812        assert_eq!(id.value(1), 2);
1813        Ok(())
1814    }
1815
1816    #[test]
1817    fn ocf_projection_with_reader_schema_alias_and_default()
1818    -> Result<(), Box<dyn std::error::Error>> {
1819        // Writer: { id: long, name: string }
1820        let writer_schema = Schema::new(vec![
1821            Field::new("id", DataType::Int64, false),
1822            Field::new("name", DataType::Utf8, false),
1823        ]);
1824        let batch = RecordBatch::try_new(
1825            Arc::new(writer_schema.clone()),
1826            vec![
1827                Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef,
1828                Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
1829            ],
1830        )?;
1831        let bytes = write_ocf(&writer_schema, &[batch]);
1832        // Reader adds alias + default field:
1833        //  - rename `name` -> `full_name` via aliases
1834        //  - add `is_active` with default true
1835        let reader_json = r#"
1836    {
1837      "type": "record",
1838      "name": "topLevelRecord",
1839      "fields": [
1840        { "name": "id", "type": "long" },
1841        { "name": "full_name", "type": ["null","string"], "aliases": ["name"], "default": null },
1842        { "name": "is_active", "type": "boolean", "default": true }
1843      ]
1844    }"#;
1845        // Project only [full_name, is_active] (indices relative to the reader schema)
1846        let mut reader = ReaderBuilder::new()
1847            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
1848            .with_projection(vec![1, 2])
1849            .build(Cursor::new(bytes))?;
1850        let out = reader.next().unwrap()?;
1851        assert_eq!(out.num_columns(), 2);
1852        assert_eq!(out.schema().field(0).name(), "full_name");
1853        assert_eq!(out.schema().field(1).name(), "is_active");
1854        let full_name = out.column(0).as_string::<i32>();
1855        assert_eq!(full_name.value(0), "a");
1856        assert_eq!(full_name.value(1), "b");
1857        let is_active = out.column(1).as_boolean();
1858        assert!(is_active.value(0));
1859        assert!(is_active.value(1));
1860        Ok(())
1861    }
1862
1863    #[test]
1864    fn projection_errors_out_of_bounds_and_duplicate() -> Result<(), Box<dyn std::error::Error>> {
1865        let writer_schema = Schema::new(vec![
1866            Field::new("a", DataType::Int32, false),
1867            Field::new("b", DataType::Int32, false),
1868        ]);
1869        let batch = RecordBatch::try_new(
1870            Arc::new(writer_schema.clone()),
1871            vec![
1872                Arc::new(Int32Array::from(vec![1])) as ArrayRef,
1873                Arc::new(Int32Array::from(vec![2])) as ArrayRef,
1874            ],
1875        )?;
1876        let bytes = write_ocf(&writer_schema, &[batch]);
1877        let err = ReaderBuilder::new()
1878            .with_projection(vec![2])
1879            .build(Cursor::new(bytes.clone()))
1880            .unwrap_err();
1881        assert!(matches!(err, ArrowError::AvroError(_)));
1882        assert!(err.to_string().contains("out of bounds"));
1883        let err = ReaderBuilder::new()
1884            .with_projection(vec![0, 0])
1885            .build(Cursor::new(bytes))
1886            .unwrap_err();
1887        assert!(matches!(err, ArrowError::AvroError(_)));
1888        assert!(err.to_string().contains("Duplicate projection index"));
1889        Ok(())
1890    }
1891
1892    #[test]
1893    #[cfg(feature = "snappy")]
1894    fn test_alltypes_plain_with_projection_and_reader_schema() {
1895        use std::fs::File;
1896        use std::io::BufReader;
1897        let path = arrow_test_data("avro/alltypes_plain.avro");
1898        // Build a reader schema that selects [double_col, id, tinyint_col] in that order
1899        let reader_schema = make_reader_schema_with_selected_fields_in_order(
1900            &path,
1901            &["double_col", "id", "tinyint_col"],
1902        );
1903        let file = File::open(&path).expect("open avro/alltypes_plain.avro");
1904        let reader = ReaderBuilder::new()
1905            .with_batch_size(1024)
1906            .with_reader_schema(reader_schema)
1907            .with_projection(vec![1, 2]) // Select indices 1 and 2 from reader schema: [id, tinyint_col]
1908            .build(BufReader::new(file))
1909            .expect("build reader with projection and reader schema");
1910        let schema = reader.schema();
1911        // Verify the projected schema has exactly 2 fields in the correct order
1912        assert_eq!(schema.fields().len(), 2);
1913        assert_eq!(schema.field(0).name(), "id");
1914        assert_eq!(schema.field(1).name(), "tinyint_col");
1915        let batches: Vec<RecordBatch> = reader.collect::<Result<Vec<_>, _>>().unwrap();
1916        assert_eq!(batches.len(), 1);
1917        let batch = &batches[0];
1918        assert_eq!(batch.num_rows(), 8);
1919        assert_eq!(batch.num_columns(), 2);
1920        // Build expected batch with exact values from alltypes_plain.avro:
1921        // - id values: [4, 5, 6, 7, 2, 3, 0, 1]
1922        // - tinyint_col values: [0, 1, 0, 1, 0, 1, 0, 1] (i.e., row_index % 2)
1923        let expected = RecordBatch::try_from_iter_with_nullable([
1924            (
1925                "id",
1926                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as ArrayRef,
1927                true,
1928            ),
1929            (
1930                "tinyint_col",
1931                Arc::new(Int32Array::from(vec![0, 1, 0, 1, 0, 1, 0, 1])) as ArrayRef,
1932                true,
1933            ),
1934        ])
1935        .unwrap();
1936        assert_eq!(
1937            batch, &expected,
1938            "Projected batch mismatch for alltypes_plain.avro with reader schema and projection [1, 2]"
1939        );
1940    }
1941
1942    #[test]
1943    #[cfg(feature = "snappy")]
1944    fn test_alltypes_plain_with_projection() {
1945        use std::fs::File;
1946        use std::io::BufReader;
1947        let path = arrow_test_data("avro/alltypes_plain.avro");
1948        let file = File::open(&path).expect("open avro/alltypes_plain.avro");
1949        let reader = ReaderBuilder::new()
1950            .with_batch_size(1024)
1951            .with_projection(vec![2, 0, 5])
1952            .build(BufReader::new(file))
1953            .expect("build reader with projection");
1954        let schema = reader.schema();
1955        assert_eq!(schema.fields().len(), 3);
1956        assert_eq!(schema.field(0).name(), "tinyint_col");
1957        assert_eq!(schema.field(1).name(), "id");
1958        assert_eq!(schema.field(2).name(), "bigint_col");
1959        let batches: Vec<RecordBatch> = reader.collect::<Result<Vec<_>, _>>().unwrap();
1960        assert_eq!(batches.len(), 1);
1961        let batch = &batches[0];
1962        assert_eq!(batch.num_rows(), 8);
1963        assert_eq!(batch.num_columns(), 3);
1964        let expected = RecordBatch::try_from_iter_with_nullable([
1965            (
1966                "tinyint_col",
1967                Arc::new(Int32Array::from(vec![0, 1, 0, 1, 0, 1, 0, 1])) as ArrayRef,
1968                true,
1969            ),
1970            (
1971                "id",
1972                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as ArrayRef,
1973                true,
1974            ),
1975            (
1976                "bigint_col",
1977                Arc::new(Int64Array::from(vec![0, 10, 0, 10, 0, 10, 0, 10])) as ArrayRef,
1978                true,
1979            ),
1980        ])
1981        .unwrap();
1982        assert_eq!(
1983            batch, &expected,
1984            "Projected batch mismatch for alltypes_plain.avro with projection [2, 0, 5]"
1985        );
1986    }
1987
1988    #[test]
1989    fn writer_string_reader_nullable_with_alias() -> Result<(), Box<dyn std::error::Error>> {
1990        let writer_schema = Schema::new(vec![
1991            Field::new("id", DataType::Int64, false),
1992            Field::new("name", DataType::Utf8, false),
1993        ]);
1994        let batch = RecordBatch::try_new(
1995            Arc::new(writer_schema.clone()),
1996            vec![
1997                Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef,
1998                Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
1999            ],
2000        )?;
2001        let bytes = write_ocf(&writer_schema, &[batch]);
2002        let reader_json = r#"
2003    {
2004      "type": "record",
2005      "name": "topLevelRecord",
2006      "fields": [
2007        { "name": "id", "type": "long" },
2008        { "name": "full_name", "type": ["null","string"], "aliases": ["name"], "default": null },
2009        { "name": "is_active", "type": "boolean", "default": true }
2010      ]
2011    }"#;
2012        let mut reader = ReaderBuilder::new()
2013            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
2014            .build(Cursor::new(bytes))?;
2015        let out = reader.next().unwrap()?;
2016        let full_name = out.column(1).as_string::<i32>();
2017        assert_eq!(full_name.value(0), "a");
2018        assert_eq!(full_name.value(1), "b");
2019        Ok(())
2020    }
2021
2022    #[test]
2023    fn writer_string_reader_string_null_order_second() -> Result<(), Box<dyn std::error::Error>> {
2024        // Writer: { name: string }
2025        let writer_schema = Schema::new(vec![Field::new("name", DataType::Utf8, false)]);
2026        let batch = RecordBatch::try_new(
2027            Arc::new(writer_schema.clone()),
2028            vec![Arc::new(StringArray::from(vec!["x", "y"])) as ArrayRef],
2029        )?;
2030        let bytes = write_ocf(&writer_schema, &[batch]);
2031
2032        // Reader: ["string","null"] (NullSecond)
2033        let reader_json = r#"
2034    {
2035      "type":"record", "name":"topLevelRecord",
2036      "fields":[ { "name":"name", "type":["string","null"], "default":"x" } ]
2037    }"#;
2038
2039        let mut reader = ReaderBuilder::new()
2040            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
2041            .build(Cursor::new(bytes))?;
2042
2043        let out = reader.next().unwrap()?;
2044        assert_eq!(out.num_rows(), 2);
2045
2046        // Should decode as non-null strings (writer non-union -> reader union)
2047        let name = out.column(0).as_string::<i32>();
2048        assert_eq!(name.value(0), "x");
2049        assert_eq!(name.value(1), "y");
2050
2051        Ok(())
2052    }
2053
2054    #[test]
2055    fn promotion_writer_int_reader_nullable_long() -> Result<(), Box<dyn std::error::Error>> {
2056        // Writer: { v: int }
2057        let writer_schema = Schema::new(vec![Field::new("v", DataType::Int32, false)]);
2058        let batch = RecordBatch::try_new(
2059            Arc::new(writer_schema.clone()),
2060            vec![Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef],
2061        )?;
2062        let bytes = write_ocf(&writer_schema, &[batch]);
2063
2064        // Reader: { v: ["null","long"] }
2065        let reader_json = r#"
2066    {
2067      "type":"record", "name":"topLevelRecord",
2068      "fields":[ { "name":"v", "type":["null","long"], "default": null } ]
2069    }"#;
2070
2071        let mut reader = ReaderBuilder::new()
2072            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
2073            .build(Cursor::new(bytes))?;
2074
2075        let out = reader.next().unwrap()?;
2076        assert_eq!(out.num_rows(), 3);
2077
2078        // Should have promoted to Int64 and be non-null (no union tag in writer)
2079        let v = out
2080            .column(0)
2081            .as_primitive::<arrow_array::types::Int64Type>();
2082        assert_eq!(v.values(), &[1, 2, 3]);
2083        assert!(
2084            out.column(0).nulls().is_none(),
2085            "expected no validity bitmap for all-valid column"
2086        );
2087
2088        Ok(())
2089    }
2090
2091    #[test]
2092    fn test_alltypes_schema_promotion_mixed() {
2093        for file in files() {
2094            let file = arrow_test_data(file);
2095            let mut promotions: HashMap<&str, &str> = HashMap::new();
2096            promotions.insert("id", "long");
2097            promotions.insert("tinyint_col", "float");
2098            promotions.insert("smallint_col", "double");
2099            promotions.insert("int_col", "double");
2100            promotions.insert("bigint_col", "double");
2101            promotions.insert("float_col", "double");
2102            promotions.insert("date_string_col", "string");
2103            promotions.insert("string_col", "string");
2104            let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2105            let batch = read_alltypes_with_reader_schema(&file, reader_schema);
2106            let expected = RecordBatch::try_from_iter_with_nullable([
2107                (
2108                    "id",
2109                    Arc::new(Int64Array::from(vec![4i64, 5, 6, 7, 2, 3, 0, 1])) as _,
2110                    true,
2111                ),
2112                (
2113                    "bool_col",
2114                    Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
2115                    true,
2116                ),
2117                (
2118                    "tinyint_col",
2119                    Arc::new(Float32Array::from_iter_values(
2120                        (0..8).map(|x| (x % 2) as f32),
2121                    )) as _,
2122                    true,
2123                ),
2124                (
2125                    "smallint_col",
2126                    Arc::new(Float64Array::from_iter_values(
2127                        (0..8).map(|x| (x % 2) as f64),
2128                    )) as _,
2129                    true,
2130                ),
2131                (
2132                    "int_col",
2133                    Arc::new(Float64Array::from_iter_values(
2134                        (0..8).map(|x| (x % 2) as f64),
2135                    )) as _,
2136                    true,
2137                ),
2138                (
2139                    "bigint_col",
2140                    Arc::new(Float64Array::from_iter_values(
2141                        (0..8).map(|x| ((x % 2) * 10) as f64),
2142                    )) as _,
2143                    true,
2144                ),
2145                (
2146                    "float_col",
2147                    Arc::new(Float64Array::from_iter_values(
2148                        (0..8).map(|x| ((x % 2) as f32 * 1.1f32) as f64),
2149                    )) as _,
2150                    true,
2151                ),
2152                (
2153                    "double_col",
2154                    Arc::new(Float64Array::from_iter_values(
2155                        (0..8).map(|x| (x % 2) as f64 * 10.1),
2156                    )) as _,
2157                    true,
2158                ),
2159                (
2160                    "date_string_col",
2161                    Arc::new(StringArray::from(vec![
2162                        "03/01/09", "03/01/09", "04/01/09", "04/01/09", "02/01/09", "02/01/09",
2163                        "01/01/09", "01/01/09",
2164                    ])) as _,
2165                    true,
2166                ),
2167                (
2168                    "string_col",
2169                    Arc::new(StringArray::from(
2170                        (0..8)
2171                            .map(|x| if x % 2 == 0 { "0" } else { "1" })
2172                            .collect::<Vec<_>>(),
2173                    )) as _,
2174                    true,
2175                ),
2176                (
2177                    "timestamp_col",
2178                    Arc::new(
2179                        TimestampMicrosecondArray::from_iter_values([
2180                            1235865600000000, // 2009-03-01T00:00:00.000
2181                            1235865660000000, // 2009-03-01T00:01:00.000
2182                            1238544000000000, // 2009-04-01T00:00:00.000
2183                            1238544060000000, // 2009-04-01T00:01:00.000
2184                            1233446400000000, // 2009-02-01T00:00:00.000
2185                            1233446460000000, // 2009-02-01T00:01:00.000
2186                            1230768000000000, // 2009-01-01T00:00:00.000
2187                            1230768060000000, // 2009-01-01T00:01:00.000
2188                        ])
2189                        .with_timezone("+00:00"),
2190                    ) as _,
2191                    true,
2192                ),
2193            ])
2194            .unwrap();
2195            assert_eq!(batch, expected, "mismatch for file {file}");
2196        }
2197    }
2198
2199    #[test]
2200    fn test_alltypes_schema_promotion_long_to_float_only() {
2201        for file in files() {
2202            let file = arrow_test_data(file);
2203            let mut promotions: HashMap<&str, &str> = HashMap::new();
2204            promotions.insert("bigint_col", "float");
2205            let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2206            let batch = read_alltypes_with_reader_schema(&file, reader_schema);
2207            let expected = RecordBatch::try_from_iter_with_nullable([
2208                (
2209                    "id",
2210                    Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
2211                    true,
2212                ),
2213                (
2214                    "bool_col",
2215                    Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
2216                    true,
2217                ),
2218                (
2219                    "tinyint_col",
2220                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2221                    true,
2222                ),
2223                (
2224                    "smallint_col",
2225                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2226                    true,
2227                ),
2228                (
2229                    "int_col",
2230                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2231                    true,
2232                ),
2233                (
2234                    "bigint_col",
2235                    Arc::new(Float32Array::from_iter_values(
2236                        (0..8).map(|x| ((x % 2) * 10) as f32),
2237                    )) as _,
2238                    true,
2239                ),
2240                (
2241                    "float_col",
2242                    Arc::new(Float32Array::from_iter_values(
2243                        (0..8).map(|x| (x % 2) as f32 * 1.1),
2244                    )) as _,
2245                    true,
2246                ),
2247                (
2248                    "double_col",
2249                    Arc::new(Float64Array::from_iter_values(
2250                        (0..8).map(|x| (x % 2) as f64 * 10.1),
2251                    )) as _,
2252                    true,
2253                ),
2254                (
2255                    "date_string_col",
2256                    Arc::new(BinaryArray::from_iter_values([
2257                        [48, 51, 47, 48, 49, 47, 48, 57],
2258                        [48, 51, 47, 48, 49, 47, 48, 57],
2259                        [48, 52, 47, 48, 49, 47, 48, 57],
2260                        [48, 52, 47, 48, 49, 47, 48, 57],
2261                        [48, 50, 47, 48, 49, 47, 48, 57],
2262                        [48, 50, 47, 48, 49, 47, 48, 57],
2263                        [48, 49, 47, 48, 49, 47, 48, 57],
2264                        [48, 49, 47, 48, 49, 47, 48, 57],
2265                    ])) as _,
2266                    true,
2267                ),
2268                (
2269                    "string_col",
2270                    Arc::new(BinaryArray::from_iter_values((0..8).map(|x| [48 + x % 2]))) as _,
2271                    true,
2272                ),
2273                (
2274                    "timestamp_col",
2275                    Arc::new(
2276                        TimestampMicrosecondArray::from_iter_values([
2277                            1235865600000000, // 2009-03-01T00:00:00.000
2278                            1235865660000000, // 2009-03-01T00:01:00.000
2279                            1238544000000000, // 2009-04-01T00:00:00.000
2280                            1238544060000000, // 2009-04-01T00:01:00.000
2281                            1233446400000000, // 2009-02-01T00:00:00.000
2282                            1233446460000000, // 2009-02-01T00:01:00.000
2283                            1230768000000000, // 2009-01-01T00:00:00.000
2284                            1230768060000000, // 2009-01-01T00:01:00.000
2285                        ])
2286                        .with_timezone("+00:00"),
2287                    ) as _,
2288                    true,
2289                ),
2290            ])
2291            .unwrap();
2292            assert_eq!(batch, expected, "mismatch for file {file}");
2293        }
2294    }
2295
2296    #[test]
2297    fn test_alltypes_schema_promotion_bytes_to_string_only() {
2298        for file in files() {
2299            let file = arrow_test_data(file);
2300            let mut promotions: HashMap<&str, &str> = HashMap::new();
2301            promotions.insert("date_string_col", "string");
2302            promotions.insert("string_col", "string");
2303            let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2304            let batch = read_alltypes_with_reader_schema(&file, reader_schema);
2305            let expected = RecordBatch::try_from_iter_with_nullable([
2306                (
2307                    "id",
2308                    Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
2309                    true,
2310                ),
2311                (
2312                    "bool_col",
2313                    Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
2314                    true,
2315                ),
2316                (
2317                    "tinyint_col",
2318                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2319                    true,
2320                ),
2321                (
2322                    "smallint_col",
2323                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2324                    true,
2325                ),
2326                (
2327                    "int_col",
2328                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2329                    true,
2330                ),
2331                (
2332                    "bigint_col",
2333                    Arc::new(Int64Array::from_iter_values((0..8).map(|x| (x % 2) * 10))) as _,
2334                    true,
2335                ),
2336                (
2337                    "float_col",
2338                    Arc::new(Float32Array::from_iter_values(
2339                        (0..8).map(|x| (x % 2) as f32 * 1.1),
2340                    )) as _,
2341                    true,
2342                ),
2343                (
2344                    "double_col",
2345                    Arc::new(Float64Array::from_iter_values(
2346                        (0..8).map(|x| (x % 2) as f64 * 10.1),
2347                    )) as _,
2348                    true,
2349                ),
2350                (
2351                    "date_string_col",
2352                    Arc::new(StringArray::from(vec![
2353                        "03/01/09", "03/01/09", "04/01/09", "04/01/09", "02/01/09", "02/01/09",
2354                        "01/01/09", "01/01/09",
2355                    ])) as _,
2356                    true,
2357                ),
2358                (
2359                    "string_col",
2360                    Arc::new(StringArray::from(
2361                        (0..8)
2362                            .map(|x| if x % 2 == 0 { "0" } else { "1" })
2363                            .collect::<Vec<_>>(),
2364                    )) as _,
2365                    true,
2366                ),
2367                (
2368                    "timestamp_col",
2369                    Arc::new(
2370                        TimestampMicrosecondArray::from_iter_values([
2371                            1235865600000000, // 2009-03-01T00:00:00.000
2372                            1235865660000000, // 2009-03-01T00:01:00.000
2373                            1238544000000000, // 2009-04-01T00:00:00.000
2374                            1238544060000000, // 2009-04-01T00:01:00.000
2375                            1233446400000000, // 2009-02-01T00:00:00.000
2376                            1233446460000000, // 2009-02-01T00:01:00.000
2377                            1230768000000000, // 2009-01-01T00:00:00.000
2378                            1230768060000000, // 2009-01-01T00:01:00.000
2379                        ])
2380                        .with_timezone("+00:00"),
2381                    ) as _,
2382                    true,
2383                ),
2384            ])
2385            .unwrap();
2386            assert_eq!(batch, expected, "mismatch for file {file}");
2387        }
2388    }
2389
2390    #[test]
2391    // TODO: avoid requiring snappy for this file
2392    #[cfg(feature = "snappy")]
2393    fn test_alltypes_illegal_promotion_bool_to_double_errors() {
2394        let file = arrow_test_data("avro/alltypes_plain.avro");
2395        let mut promotions: HashMap<&str, &str> = HashMap::new();
2396        promotions.insert("bool_col", "double"); // illegal
2397        let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2398        let file_handle = File::open(&file).unwrap();
2399        let result = ReaderBuilder::new()
2400            .with_reader_schema(reader_schema)
2401            .build(BufReader::new(file_handle));
2402        let err = result.expect_err("expected illegal promotion to error");
2403        let msg = err.to_string();
2404        assert!(
2405            msg.contains("Illegal promotion") || msg.contains("illegal promotion"),
2406            "unexpected error: {msg}"
2407        );
2408    }
2409
2410    #[test]
2411    fn test_simple_enum_with_reader_schema_mapping() {
2412        let file = arrow_test_data("avro/simple_enum.avro");
2413        let mut remap: HashMap<&str, Vec<&str>> = HashMap::new();
2414        remap.insert("f1", vec!["d", "c", "b", "a"]);
2415        remap.insert("f2", vec!["h", "g", "f", "e"]);
2416        remap.insert("f3", vec!["k", "i", "j"]);
2417        let reader_schema = make_reader_schema_with_enum_remap(&file, &remap);
2418        let actual = read_alltypes_with_reader_schema(&file, reader_schema);
2419        let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8));
2420        // f1
2421        let f1_keys = Int32Array::from(vec![3, 2, 1, 0]);
2422        let f1_vals = StringArray::from(vec!["d", "c", "b", "a"]);
2423        let f1 = DictionaryArray::<Int32Type>::try_new(f1_keys, Arc::new(f1_vals)).unwrap();
2424        let mut md_f1 = HashMap::new();
2425        md_f1.insert(
2426            AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
2427            r#"["d","c","b","a"]"#.to_string(),
2428        );
2429        // New named-type metadata
2430        md_f1.insert("avro.name".to_string(), "enum1".to_string());
2431        md_f1.insert("avro.namespace".to_string(), "ns1".to_string());
2432        let f1_field = Field::new("f1", dict_type.clone(), false).with_metadata(md_f1);
2433        // f2
2434        let f2_keys = Int32Array::from(vec![1, 0, 3, 2]);
2435        let f2_vals = StringArray::from(vec!["h", "g", "f", "e"]);
2436        let f2 = DictionaryArray::<Int32Type>::try_new(f2_keys, Arc::new(f2_vals)).unwrap();
2437        let mut md_f2 = HashMap::new();
2438        md_f2.insert(
2439            AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
2440            r#"["h","g","f","e"]"#.to_string(),
2441        );
2442        // New named-type metadata
2443        md_f2.insert("avro.name".to_string(), "enum2".to_string());
2444        md_f2.insert("avro.namespace".to_string(), "ns2".to_string());
2445        let f2_field = Field::new("f2", dict_type.clone(), false).with_metadata(md_f2);
2446        // f3
2447        let f3_keys = Int32Array::from(vec![Some(2), Some(0), None, Some(1)]);
2448        let f3_vals = StringArray::from(vec!["k", "i", "j"]);
2449        let f3 = DictionaryArray::<Int32Type>::try_new(f3_keys, Arc::new(f3_vals)).unwrap();
2450        let mut md_f3 = HashMap::new();
2451        md_f3.insert(
2452            AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
2453            r#"["k","i","j"]"#.to_string(),
2454        );
2455        // New named-type metadata
2456        md_f3.insert("avro.name".to_string(), "enum3".to_string());
2457        md_f3.insert("avro.namespace".to_string(), "ns1".to_string());
2458        let f3_field = Field::new("f3", dict_type.clone(), true).with_metadata(md_f3);
2459        let expected_schema = Arc::new(Schema::new(vec![f1_field, f2_field, f3_field]));
2460        let expected = RecordBatch::try_new(
2461            expected_schema,
2462            vec![Arc::new(f1) as ArrayRef, Arc::new(f2), Arc::new(f3)],
2463        )
2464        .unwrap();
2465        assert_eq!(actual, expected);
2466    }
2467
2468    #[test]
2469    fn test_schema_store_register_lookup() {
2470        let schema_int = make_record_schema(PrimitiveType::Int);
2471        let schema_long = make_record_schema(PrimitiveType::Long);
2472        let mut store = SchemaStore::new();
2473        let fp_int = store.register(schema_int.clone()).unwrap();
2474        let fp_long = store.register(schema_long.clone()).unwrap();
2475        assert_eq!(store.lookup(&fp_int).cloned(), Some(schema_int));
2476        assert_eq!(store.lookup(&fp_long).cloned(), Some(schema_long));
2477        assert_eq!(store.fingerprint_algorithm(), FingerprintAlgorithm::Rabin);
2478    }
2479
2480    #[test]
2481    fn test_unknown_fingerprint_is_error() {
2482        let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store();
2483        let unknown_fp = Fingerprint::Rabin(0xDEAD_BEEF_DEAD_BEEF);
2484        let prefix = make_prefix(unknown_fp);
2485        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2486        let err = decoder.decode(&prefix).expect_err("decode should error");
2487        let msg = err.to_string();
2488        assert!(
2489            msg.contains("Unknown fingerprint"),
2490            "unexpected message: {msg}"
2491        );
2492    }
2493
2494    #[test]
2495    fn test_handle_prefix_incomplete_magic() {
2496        let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store();
2497        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2498        let buf = &SINGLE_OBJECT_MAGIC[..1];
2499        let res = decoder.handle_prefix(buf).unwrap();
2500        assert_eq!(res, Some(0));
2501        assert!(decoder.pending_schema.is_none());
2502    }
2503
2504    #[test]
2505    fn test_handle_prefix_magic_mismatch() {
2506        let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store();
2507        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2508        let buf = [0xFFu8, 0x00u8, 0x01u8];
2509        let res = decoder.handle_prefix(&buf).unwrap();
2510        assert!(res.is_none());
2511    }
2512
2513    #[test]
2514    fn test_handle_prefix_incomplete_fingerprint() {
2515        let (store, fp_int, fp_long, _schema_int, schema_long) = make_two_schema_store();
2516        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2517        let long_bytes = match fp_long {
2518            Fingerprint::Rabin(v) => v.to_le_bytes(),
2519            Fingerprint::Id(id) => panic!("expected Rabin fingerprint, got ({id})"),
2520            Fingerprint::Id64(id) => panic!("expected Rabin fingerprint, got ({id})"),
2521            #[cfg(feature = "md5")]
2522            Fingerprint::MD5(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2523            #[cfg(feature = "sha256")]
2524            Fingerprint::SHA256(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2525        };
2526        let mut buf = Vec::from(SINGLE_OBJECT_MAGIC);
2527        buf.extend_from_slice(&long_bytes[..4]);
2528        let res = decoder.handle_prefix(&buf).unwrap();
2529        assert_eq!(res, Some(0));
2530        assert!(decoder.pending_schema.is_none());
2531    }
2532
2533    #[test]
2534    fn test_handle_prefix_valid_prefix_switches_schema() {
2535        let (store, fp_int, fp_long, _schema_int, schema_long) = make_two_schema_store();
2536        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2537        let writer_schema_long = schema_long.schema().unwrap();
2538        let root_long = AvroFieldBuilder::new(&writer_schema_long).build().unwrap();
2539        let long_decoder = RecordDecoder::try_new_with_options(root_long.data_type()).unwrap();
2540        let _ = decoder.cache.insert(fp_long, long_decoder);
2541        let mut buf = Vec::from(SINGLE_OBJECT_MAGIC);
2542        match fp_long {
2543            Fingerprint::Rabin(v) => buf.extend_from_slice(&v.to_le_bytes()),
2544            Fingerprint::Id(id) => panic!("expected Rabin fingerprint, got ({id})"),
2545            Fingerprint::Id64(id) => panic!("expected Rabin fingerprint, got ({id})"),
2546            #[cfg(feature = "md5")]
2547            Fingerprint::MD5(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2548            #[cfg(feature = "sha256")]
2549            Fingerprint::SHA256(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2550        }
2551        let consumed = decoder.handle_prefix(&buf).unwrap().unwrap();
2552        assert_eq!(consumed, buf.len());
2553        assert!(decoder.pending_schema.is_some());
2554        assert_eq!(decoder.pending_schema.as_ref().unwrap().0, fp_long);
2555    }
2556
2557    #[test]
2558    fn test_decoder_projection_multiple_writer_schemas_no_reader_schema()
2559    -> Result<(), Box<dyn std::error::Error>> {
2560        // Two writer schemas with different shapes
2561        let writer_v1 = AvroSchema::new(
2562            r#"{"type":"record","name":"E","fields":[{"name":"a","type":"int"},{"name":"b","type":"string"}]}"#
2563                .to_string(),
2564        );
2565        let writer_v2 = AvroSchema::new(
2566            r#"{"type":"record","name":"E","fields":[{"name":"a","type":"long"},{"name":"b","type":"string"},{"name":"c","type":"int"}]}"#
2567                .to_string(),
2568        );
2569        let mut store = SchemaStore::new();
2570        let fp1 = store.register(writer_v1)?;
2571        let fp2 = store.register(writer_v2)?;
2572        let mut decoder = ReaderBuilder::new()
2573            .with_writer_schema_store(store)
2574            .with_active_fingerprint(fp1)
2575            .with_batch_size(8)
2576            .with_projection(vec![1])
2577            .build_decoder()?;
2578        // Message for v1: {a:1, b:"x"}
2579        let mut msg1 = make_prefix(fp1);
2580        msg1.extend_from_slice(&encode_zigzag(1)); // a = 1
2581        msg1.push((1u8) << 1);
2582        msg1.extend_from_slice(b"x");
2583        // Message for v2: {a:2, b:"y", c:7}
2584        let mut msg2 = make_prefix(fp2);
2585        msg2.extend_from_slice(&encode_zigzag(2)); // a = 2
2586        msg2.push((1u8) << 1);
2587        msg2.extend_from_slice(b"y");
2588        msg2.extend_from_slice(&encode_zigzag(7)); // c = 7
2589        decoder.decode(&msg1)?;
2590        let batch1 = decoder.flush()?.expect("batch1");
2591        assert_eq!(batch1.num_columns(), 1);
2592        assert_eq!(batch1.schema().field(0).name(), "b");
2593        let b1 = batch1.column(0).as_string::<i32>();
2594        assert_eq!(b1.value(0), "x");
2595        decoder.decode(&msg2)?;
2596        let batch2 = decoder.flush()?.expect("batch2");
2597        assert_eq!(batch2.num_columns(), 1);
2598        assert_eq!(batch2.schema().field(0).name(), "b");
2599        let b2 = batch2.column(0).as_string::<i32>();
2600        assert_eq!(b2.value(0), "y");
2601        Ok(())
2602    }
2603
2604    #[test]
2605    fn test_two_messages_same_schema() {
2606        let writer_schema = make_value_schema(PrimitiveType::Int);
2607        let reader_schema = writer_schema.clone();
2608        let mut store = SchemaStore::new();
2609        let fp = store.register(writer_schema).unwrap();
2610        let msg1 = make_message(fp, 42);
2611        let msg2 = make_message(fp, 11);
2612        let input = [msg1.clone(), msg2.clone()].concat();
2613        let mut decoder = ReaderBuilder::new()
2614            .with_batch_size(8)
2615            .with_reader_schema(reader_schema.clone())
2616            .with_writer_schema_store(store)
2617            .with_active_fingerprint(fp)
2618            .build_decoder()
2619            .unwrap();
2620        let _ = decoder.decode(&input).unwrap();
2621        let batch = decoder.flush().unwrap().expect("batch");
2622        assert_eq!(batch.num_rows(), 2);
2623        let col = batch
2624            .column(0)
2625            .as_any()
2626            .downcast_ref::<Int32Array>()
2627            .unwrap();
2628        assert_eq!(col.value(0), 42);
2629        assert_eq!(col.value(1), 11);
2630    }
2631
2632    #[test]
2633    fn test_two_messages_schema_switch() {
2634        let w_int = make_value_schema(PrimitiveType::Int);
2635        let w_long = make_value_schema(PrimitiveType::Long);
2636        let mut store = SchemaStore::new();
2637        let fp_int = store.register(w_int).unwrap();
2638        let fp_long = store.register(w_long).unwrap();
2639        let msg_int = make_message(fp_int, 1);
2640        let msg_long = make_message(fp_long, 123456789_i64);
2641        let mut decoder = ReaderBuilder::new()
2642            .with_batch_size(8)
2643            .with_writer_schema_store(store)
2644            .with_active_fingerprint(fp_int)
2645            .build_decoder()
2646            .unwrap();
2647        let _ = decoder.decode(&msg_int).unwrap();
2648        let batch1 = decoder.flush().unwrap().expect("batch1");
2649        assert_eq!(batch1.num_rows(), 1);
2650        assert_eq!(
2651            batch1
2652                .column(0)
2653                .as_any()
2654                .downcast_ref::<Int32Array>()
2655                .unwrap()
2656                .value(0),
2657            1
2658        );
2659        let _ = decoder.decode(&msg_long).unwrap();
2660        let batch2 = decoder.flush().unwrap().expect("batch2");
2661        assert_eq!(batch2.num_rows(), 1);
2662        assert_eq!(
2663            batch2
2664                .column(0)
2665                .as_any()
2666                .downcast_ref::<Int64Array>()
2667                .unwrap()
2668                .value(0),
2669            123456789_i64
2670        );
2671    }
2672
2673    #[test]
2674    fn test_two_messages_same_schema_id() {
2675        let writer_schema = make_value_schema(PrimitiveType::Int);
2676        let reader_schema = writer_schema.clone();
2677        let id = 100u32;
2678        // Set up store with None fingerprint algorithm and register schema by id
2679        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
2680        let _ = store
2681            .set(Fingerprint::Id(id), writer_schema.clone())
2682            .expect("set id schema");
2683        let msg1 = make_message_id(id, 21);
2684        let msg2 = make_message_id(id, 22);
2685        let input = [msg1.clone(), msg2.clone()].concat();
2686        let mut decoder = ReaderBuilder::new()
2687            .with_batch_size(8)
2688            .with_reader_schema(reader_schema)
2689            .with_writer_schema_store(store)
2690            .with_active_fingerprint(Fingerprint::Id(id))
2691            .build_decoder()
2692            .unwrap();
2693        let _ = decoder.decode(&input).unwrap();
2694        let batch = decoder.flush().unwrap().expect("batch");
2695        assert_eq!(batch.num_rows(), 2);
2696        let col = batch
2697            .column(0)
2698            .as_any()
2699            .downcast_ref::<Int32Array>()
2700            .unwrap();
2701        assert_eq!(col.value(0), 21);
2702        assert_eq!(col.value(1), 22);
2703    }
2704
2705    #[test]
2706    fn test_unknown_id_fingerprint_is_error() {
2707        let writer_schema = make_value_schema(PrimitiveType::Int);
2708        let id_known = 7u32;
2709        let id_unknown = 9u32;
2710        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
2711        let _ = store
2712            .set(Fingerprint::Id(id_known), writer_schema.clone())
2713            .expect("set id schema");
2714        let mut decoder = ReaderBuilder::new()
2715            .with_batch_size(8)
2716            .with_reader_schema(writer_schema)
2717            .with_writer_schema_store(store)
2718            .with_active_fingerprint(Fingerprint::Id(id_known))
2719            .build_decoder()
2720            .unwrap();
2721        let prefix = make_id_prefix(id_unknown, 0);
2722        let err = decoder.decode(&prefix).expect_err("decode should error");
2723        let msg = err.to_string();
2724        assert!(
2725            msg.contains("Unknown fingerprint"),
2726            "unexpected message: {msg}"
2727        );
2728    }
2729
2730    #[test]
2731    fn test_handle_prefix_id_incomplete_magic() {
2732        let writer_schema = make_value_schema(PrimitiveType::Int);
2733        let id = 5u32;
2734        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
2735        let _ = store
2736            .set(Fingerprint::Id(id), writer_schema.clone())
2737            .expect("set id schema");
2738        let mut decoder = ReaderBuilder::new()
2739            .with_batch_size(8)
2740            .with_reader_schema(writer_schema)
2741            .with_writer_schema_store(store)
2742            .with_active_fingerprint(Fingerprint::Id(id))
2743            .build_decoder()
2744            .unwrap();
2745        let buf = &CONFLUENT_MAGIC[..0]; // empty incomplete magic
2746        let res = decoder.handle_prefix(buf).unwrap();
2747        assert_eq!(res, Some(0));
2748        assert!(decoder.pending_schema.is_none());
2749    }
2750
2751    #[test]
2752    fn test_two_messages_same_schema_id64() {
2753        let writer_schema = make_value_schema(PrimitiveType::Int);
2754        let reader_schema = writer_schema.clone();
2755        let id = 100u64;
2756        // Set up store with None fingerprint algorithm and register schema by id
2757        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id64);
2758        let _ = store
2759            .set(Fingerprint::Id64(id), writer_schema.clone())
2760            .expect("set id schema");
2761        let msg1 = make_message_id64(id, 21);
2762        let msg2 = make_message_id64(id, 22);
2763        let input = [msg1.clone(), msg2.clone()].concat();
2764        let mut decoder = ReaderBuilder::new()
2765            .with_batch_size(8)
2766            .with_reader_schema(reader_schema)
2767            .with_writer_schema_store(store)
2768            .with_active_fingerprint(Fingerprint::Id64(id))
2769            .build_decoder()
2770            .unwrap();
2771        let _ = decoder.decode(&input).unwrap();
2772        let batch = decoder.flush().unwrap().expect("batch");
2773        assert_eq!(batch.num_rows(), 2);
2774        let col = batch
2775            .column(0)
2776            .as_any()
2777            .downcast_ref::<Int32Array>()
2778            .unwrap();
2779        assert_eq!(col.value(0), 21);
2780        assert_eq!(col.value(1), 22);
2781    }
2782
2783    #[test]
2784    fn test_decode_stream_with_schema() {
2785        struct TestCase<'a> {
2786            name: &'a str,
2787            schema: &'a str,
2788            expected_error: Option<&'a str>,
2789        }
2790        let tests = vec![
2791            TestCase {
2792                name: "success",
2793                schema: r#"{"type":"record","name":"test","fields":[{"name":"f2","type":"string"}]}"#,
2794                expected_error: None,
2795            },
2796            TestCase {
2797                name: "valid schema invalid data",
2798                schema: r#"{"type":"record","name":"test","fields":[{"name":"f2","type":"long"}]}"#,
2799                expected_error: Some("did not consume all bytes"),
2800            },
2801        ];
2802        for test in tests {
2803            let avro_schema = AvroSchema::new(test.schema.to_string());
2804            let mut store = SchemaStore::new();
2805            let fp = store.register(avro_schema.clone()).unwrap();
2806            let prefix = make_prefix(fp);
2807            let record_val = "some_string";
2808            let mut body = prefix;
2809            body.push((record_val.len() as u8) << 1);
2810            body.extend_from_slice(record_val.as_bytes());
2811            let decoder_res = ReaderBuilder::new()
2812                .with_batch_size(1)
2813                .with_writer_schema_store(store)
2814                .with_active_fingerprint(fp)
2815                .build_decoder();
2816            let decoder = match decoder_res {
2817                Ok(d) => d,
2818                Err(e) => {
2819                    if let Some(expected) = test.expected_error {
2820                        assert!(
2821                            e.to_string().contains(expected),
2822                            "Test '{}' failed at build – expected '{expected}', got '{e}'",
2823                            test.name
2824                        );
2825                        continue;
2826                    } else {
2827                        panic!("Test '{}' failed during build: {e}", test.name);
2828                    }
2829                }
2830            };
2831            let stream = Box::pin(stream::once(async { Bytes::from(body) }));
2832            let decoded_stream = decode_stream(decoder, stream);
2833            let batches_result: Result<Vec<RecordBatch>, ArrowError> =
2834                block_on(decoded_stream.try_collect());
2835            match (batches_result, test.expected_error) {
2836                (Ok(batches), None) => {
2837                    let batch =
2838                        arrow::compute::concat_batches(&batches[0].schema(), &batches).unwrap();
2839                    let expected_field = Field::new("f2", DataType::Utf8, false);
2840                    let expected_schema = Arc::new(Schema::new(vec![expected_field]));
2841                    let expected_array = Arc::new(StringArray::from(vec![record_val]));
2842                    let expected_batch =
2843                        RecordBatch::try_new(expected_schema, vec![expected_array]).unwrap();
2844                    assert_eq!(batch, expected_batch, "Test '{}'", test.name);
2845                }
2846                (Err(e), Some(expected)) => {
2847                    assert!(
2848                        e.to_string().contains(expected),
2849                        "Test '{}' – expected error containing '{expected}', got '{e}'",
2850                        test.name
2851                    );
2852                }
2853                (Ok(_), Some(expected)) => {
2854                    panic!(
2855                        "Test '{}' expected failure ('{expected}') but succeeded",
2856                        test.name
2857                    );
2858                }
2859                (Err(e), None) => {
2860                    panic!("Test '{}' unexpectedly failed with '{e}'", test.name);
2861                }
2862            }
2863        }
2864    }
2865
2866    #[test]
2867    fn test_utf8view_support() {
2868        struct TestHelper;
2869        impl TestHelper {
2870            fn with_utf8view(field: &Field) -> Field {
2871                match field.data_type() {
2872                    DataType::Utf8 => {
2873                        Field::new(field.name(), DataType::Utf8View, field.is_nullable())
2874                            .with_metadata(field.metadata().clone())
2875                    }
2876                    _ => field.clone(),
2877                }
2878            }
2879        }
2880
2881        let field = TestHelper::with_utf8view(&Field::new("str_field", DataType::Utf8, false));
2882
2883        assert_eq!(field.data_type(), &DataType::Utf8View);
2884
2885        let array = StringViewArray::from(vec!["test1", "test2"]);
2886        let batch =
2887            RecordBatch::try_from_iter(vec![("str_field", Arc::new(array) as ArrayRef)]).unwrap();
2888
2889        assert!(batch.column(0).as_any().is::<StringViewArray>());
2890    }
2891
2892    fn make_reader_schema_with_default_fields(
2893        path: &str,
2894        default_fields: Vec<Value>,
2895    ) -> AvroSchema {
2896        let mut root = load_writer_schema_json(path);
2897        assert_eq!(root["type"], "record", "writer schema must be a record");
2898        root.as_object_mut()
2899            .expect("schema is a JSON object")
2900            .insert("fields".to_string(), Value::Array(default_fields));
2901        AvroSchema::new(root.to_string())
2902    }
2903
2904    #[test]
2905    fn test_schema_resolution_defaults_all_supported_types() {
2906        let path = "test/data/skippable_types.avro";
2907        let duration_default = "\u{0000}".repeat(12);
2908        let reader_schema = make_reader_schema_with_default_fields(
2909            path,
2910            vec![
2911                serde_json::json!({"name":"d_bool","type":"boolean","default":true}),
2912                serde_json::json!({"name":"d_int","type":"int","default":42}),
2913                serde_json::json!({"name":"d_long","type":"long","default":12345}),
2914                serde_json::json!({"name":"d_float","type":"float","default":1.5}),
2915                serde_json::json!({"name":"d_double","type":"double","default":2.25}),
2916                serde_json::json!({"name":"d_bytes","type":"bytes","default":"XYZ"}),
2917                serde_json::json!({"name":"d_string","type":"string","default":"hello"}),
2918                serde_json::json!({"name":"d_date","type":{"type":"int","logicalType":"date"},"default":0}),
2919                serde_json::json!({"name":"d_time_ms","type":{"type":"int","logicalType":"time-millis"},"default":1000}),
2920                serde_json::json!({"name":"d_time_us","type":{"type":"long","logicalType":"time-micros"},"default":2000}),
2921                serde_json::json!({"name":"d_ts_ms","type":{"type":"long","logicalType":"local-timestamp-millis"},"default":0}),
2922                serde_json::json!({"name":"d_ts_us","type":{"type":"long","logicalType":"local-timestamp-micros"},"default":0}),
2923                serde_json::json!({"name":"d_decimal","type":{"type":"bytes","logicalType":"decimal","precision":10,"scale":2},"default":""}),
2924                serde_json::json!({"name":"d_fixed","type":{"type":"fixed","name":"F4","size":4},"default":"ABCD"}),
2925                serde_json::json!({"name":"d_enum","type":{"type":"enum","name":"E","symbols":["A","B","C"]},"default":"A"}),
2926                serde_json::json!({"name":"d_duration","type":{"type":"fixed","name":"Dur","size":12,"logicalType":"duration"},"default":duration_default}),
2927                serde_json::json!({"name":"d_uuid","type":{"type":"string","logicalType":"uuid"},"default":"00000000-0000-0000-0000-000000000000"}),
2928                serde_json::json!({"name":"d_array","type":{"type":"array","items":"int"},"default":[1,2,3]}),
2929                serde_json::json!({"name":"d_map","type":{"type":"map","values":"long"},"default":{"a":1,"b":2}}),
2930                serde_json::json!({"name":"d_record","type":{
2931              "type":"record","name":"DefaultRec","fields":[
2932                  {"name":"x","type":"int"},
2933                  {"name":"y","type":["null","string"],"default":null}
2934              ]
2935        },"default":{"x":7}}),
2936                serde_json::json!({"name":"d_nullable_null","type":["null","int"],"default":null}),
2937                serde_json::json!({"name":"d_nullable_value","type":["int","null"],"default":123}),
2938            ],
2939        );
2940        let actual = read_alltypes_with_reader_schema(path, reader_schema);
2941        let num_rows = actual.num_rows();
2942        assert!(num_rows > 0, "skippable_types.avro should contain rows");
2943        assert_eq!(
2944            actual.num_columns(),
2945            22,
2946            "expected exactly our defaulted fields"
2947        );
2948        let mut arrays: Vec<Arc<dyn Array>> = Vec::with_capacity(22);
2949        arrays.push(Arc::new(BooleanArray::from_iter(std::iter::repeat_n(
2950            Some(true),
2951            num_rows,
2952        ))));
2953        arrays.push(Arc::new(Int32Array::from_iter_values(std::iter::repeat_n(
2954            42, num_rows,
2955        ))));
2956        arrays.push(Arc::new(Int64Array::from_iter_values(std::iter::repeat_n(
2957            12345, num_rows,
2958        ))));
2959        arrays.push(Arc::new(Float32Array::from_iter_values(
2960            std::iter::repeat_n(1.5f32, num_rows),
2961        )));
2962        arrays.push(Arc::new(Float64Array::from_iter_values(
2963            std::iter::repeat_n(2.25f64, num_rows),
2964        )));
2965        arrays.push(Arc::new(BinaryArray::from_iter_values(
2966            std::iter::repeat_n(b"XYZ".as_ref(), num_rows),
2967        )));
2968        arrays.push(Arc::new(StringArray::from_iter_values(
2969            std::iter::repeat_n("hello", num_rows),
2970        )));
2971        arrays.push(Arc::new(Date32Array::from_iter_values(
2972            std::iter::repeat_n(0, num_rows),
2973        )));
2974        arrays.push(Arc::new(Time32MillisecondArray::from_iter_values(
2975            std::iter::repeat_n(1_000, num_rows),
2976        )));
2977        arrays.push(Arc::new(Time64MicrosecondArray::from_iter_values(
2978            std::iter::repeat_n(2_000i64, num_rows),
2979        )));
2980        arrays.push(Arc::new(TimestampMillisecondArray::from_iter_values(
2981            std::iter::repeat_n(0i64, num_rows),
2982        )));
2983        arrays.push(Arc::new(TimestampMicrosecondArray::from_iter_values(
2984            std::iter::repeat_n(0i64, num_rows),
2985        )));
2986        #[cfg(feature = "small_decimals")]
2987        let decimal = Decimal64Array::from_iter_values(std::iter::repeat_n(0i64, num_rows))
2988            .with_precision_and_scale(10, 2)
2989            .unwrap();
2990        #[cfg(not(feature = "small_decimals"))]
2991        let decimal = Decimal128Array::from_iter_values(std::iter::repeat_n(0i128, num_rows))
2992            .with_precision_and_scale(10, 2)
2993            .unwrap();
2994        arrays.push(Arc::new(decimal));
2995        let fixed_iter = std::iter::repeat_n(Some(*b"ABCD"), num_rows);
2996        arrays.push(Arc::new(
2997            FixedSizeBinaryArray::try_from_sparse_iter_with_size(fixed_iter, 4).unwrap(),
2998        ));
2999        let enum_keys = Int32Array::from_iter_values(std::iter::repeat_n(0, num_rows));
3000        let enum_values = StringArray::from_iter_values(["A", "B", "C"]);
3001        let enum_arr =
3002            DictionaryArray::<Int32Type>::try_new(enum_keys, Arc::new(enum_values)).unwrap();
3003        arrays.push(Arc::new(enum_arr));
3004        let duration_values = std::iter::repeat_n(
3005            Some(IntervalMonthDayNanoType::make_value(0, 0, 0)),
3006            num_rows,
3007        );
3008        let duration_arr: IntervalMonthDayNanoArray = duration_values.collect();
3009        arrays.push(Arc::new(duration_arr));
3010        let uuid_bytes = [0u8; 16];
3011        let uuid_iter = std::iter::repeat_n(Some(uuid_bytes), num_rows);
3012        arrays.push(Arc::new(
3013            FixedSizeBinaryArray::try_from_sparse_iter_with_size(uuid_iter, 16).unwrap(),
3014        ));
3015        let item_field = Arc::new(Field::new(
3016            Field::LIST_FIELD_DEFAULT_NAME,
3017            DataType::Int32,
3018            false,
3019        ));
3020        let mut list_builder = ListBuilder::new(Int32Builder::new()).with_field(item_field);
3021        for _ in 0..num_rows {
3022            list_builder.values().append_value(1);
3023            list_builder.values().append_value(2);
3024            list_builder.values().append_value(3);
3025            list_builder.append(true);
3026        }
3027        arrays.push(Arc::new(list_builder.finish()));
3028        let values_field = Arc::new(Field::new("value", DataType::Int64, false));
3029        let mut map_builder = MapBuilder::new(
3030            Some(builder::MapFieldNames {
3031                entry: "entries".to_string(),
3032                key: "key".to_string(),
3033                value: "value".to_string(),
3034            }),
3035            StringBuilder::new(),
3036            Int64Builder::new(),
3037        )
3038        .with_values_field(values_field);
3039        for _ in 0..num_rows {
3040            let (keys, vals) = map_builder.entries();
3041            keys.append_value("a");
3042            vals.append_value(1);
3043            keys.append_value("b");
3044            vals.append_value(2);
3045            map_builder.append(true).unwrap();
3046        }
3047        arrays.push(Arc::new(map_builder.finish()));
3048        let rec_fields: Fields = Fields::from(vec![
3049            Field::new("x", DataType::Int32, false),
3050            Field::new("y", DataType::Utf8, true),
3051        ]);
3052        let mut sb = StructBuilder::new(
3053            rec_fields.clone(),
3054            vec![
3055                Box::new(Int32Builder::new()),
3056                Box::new(StringBuilder::new()),
3057            ],
3058        );
3059        for _ in 0..num_rows {
3060            sb.field_builder::<Int32Builder>(0).unwrap().append_value(7);
3061            sb.field_builder::<StringBuilder>(1).unwrap().append_null();
3062            sb.append(true);
3063        }
3064        arrays.push(Arc::new(sb.finish()));
3065        arrays.push(Arc::new(Int32Array::from_iter(std::iter::repeat_n(
3066            None::<i32>,
3067            num_rows,
3068        ))));
3069        arrays.push(Arc::new(Int32Array::from_iter_values(std::iter::repeat_n(
3070            123, num_rows,
3071        ))));
3072        let expected = RecordBatch::try_new(actual.schema(), arrays).unwrap();
3073        assert_eq!(
3074            actual, expected,
3075            "defaults should materialize correctly for all fields"
3076        );
3077    }
3078
3079    #[test]
3080    fn test_schema_resolution_default_enum_invalid_symbol_errors() {
3081        let path = "test/data/skippable_types.avro";
3082        let bad_schema = make_reader_schema_with_default_fields(
3083            path,
3084            vec![serde_json::json!({
3085                "name":"bad_enum",
3086                "type":{"type":"enum","name":"E","symbols":["A","B","C"]},
3087                "default":"Z"
3088            })],
3089        );
3090        let file = File::open(path).unwrap();
3091        let res = ReaderBuilder::new()
3092            .with_reader_schema(bad_schema)
3093            .build(BufReader::new(file));
3094        let err = res.expect_err("expected enum default validation to fail");
3095        let msg = err.to_string();
3096        let lower_msg = msg.to_lowercase();
3097        assert!(
3098            lower_msg.contains("enum")
3099                && (lower_msg.contains("symbol") || lower_msg.contains("default")),
3100            "unexpected error: {msg}"
3101        );
3102    }
3103
3104    #[test]
3105    fn test_schema_resolution_default_fixed_size_mismatch_errors() {
3106        let path = "test/data/skippable_types.avro";
3107        let bad_schema = make_reader_schema_with_default_fields(
3108            path,
3109            vec![serde_json::json!({
3110                "name":"bad_fixed",
3111                "type":{"type":"fixed","name":"F","size":4},
3112                "default":"ABC"
3113            })],
3114        );
3115        let file = File::open(path).unwrap();
3116        let res = ReaderBuilder::new()
3117            .with_reader_schema(bad_schema)
3118            .build(BufReader::new(file));
3119        let err = res.expect_err("expected fixed default validation to fail");
3120        let msg = err.to_string();
3121        let lower_msg = msg.to_lowercase();
3122        assert!(
3123            lower_msg.contains("fixed")
3124                && (lower_msg.contains("size")
3125                    || lower_msg.contains("length")
3126                    || lower_msg.contains("does not match")),
3127            "unexpected error: {msg}"
3128        );
3129    }
3130
3131    #[test]
3132    // TODO: avoid requiring snappy for this file
3133    #[cfg(feature = "snappy")]
3134    fn test_alltypes_skip_writer_fields_keep_double_only() {
3135        let file = arrow_test_data("avro/alltypes_plain.avro");
3136        let reader_schema =
3137            make_reader_schema_with_selected_fields_in_order(&file, &["double_col"]);
3138        let batch = read_alltypes_with_reader_schema(&file, reader_schema);
3139        let expected = RecordBatch::try_from_iter_with_nullable([(
3140            "double_col",
3141            Arc::new(Float64Array::from_iter_values(
3142                (0..8).map(|x| (x % 2) as f64 * 10.1),
3143            )) as _,
3144            true,
3145        )])
3146        .unwrap();
3147        assert_eq!(batch, expected);
3148    }
3149
3150    #[test]
3151    // TODO: avoid requiring snappy for this file
3152    #[cfg(feature = "snappy")]
3153    fn test_alltypes_skip_writer_fields_reorder_and_skip_many() {
3154        let file = arrow_test_data("avro/alltypes_plain.avro");
3155        let reader_schema =
3156            make_reader_schema_with_selected_fields_in_order(&file, &["timestamp_col", "id"]);
3157        let batch = read_alltypes_with_reader_schema(&file, reader_schema);
3158        let expected = RecordBatch::try_from_iter_with_nullable([
3159            (
3160                "timestamp_col",
3161                Arc::new(
3162                    TimestampMicrosecondArray::from_iter_values([
3163                        1235865600000000, // 2009-03-01T00:00:00.000
3164                        1235865660000000, // 2009-03-01T00:01:00.000
3165                        1238544000000000, // 2009-04-01T00:00:00.000
3166                        1238544060000000, // 2009-04-01T00:01:00.000
3167                        1233446400000000, // 2009-02-01T00:00:00.000
3168                        1233446460000000, // 2009-02-01T00:01:00.000
3169                        1230768000000000, // 2009-01-01T00:00:00.000
3170                        1230768060000000, // 2009-01-01T00:01:00.000
3171                    ])
3172                    .with_timezone("+00:00"),
3173                ) as _,
3174                true,
3175            ),
3176            (
3177                "id",
3178                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
3179                true,
3180            ),
3181        ])
3182        .unwrap();
3183        assert_eq!(batch, expected);
3184    }
3185
3186    #[test]
3187    fn test_skippable_types_project_each_field_individually() {
3188        let path = "test/data/skippable_types.avro";
3189        let full = read_file(path, 1024, false);
3190        let schema_full = full.schema();
3191        let num_rows = full.num_rows();
3192        let writer_json = load_writer_schema_json(path);
3193        assert_eq!(
3194            writer_json["type"], "record",
3195            "writer schema must be a record"
3196        );
3197        let fields_json = writer_json
3198            .get("fields")
3199            .and_then(|f| f.as_array())
3200            .expect("record has fields");
3201        assert_eq!(
3202            schema_full.fields().len(),
3203            fields_json.len(),
3204            "full read column count vs writer fields"
3205        );
3206        fn rebuild_list_array_with_element(
3207            col: &ArrayRef,
3208            new_elem: Arc<Field>,
3209            is_large: bool,
3210        ) -> ArrayRef {
3211            if is_large {
3212                let list = col
3213                    .as_any()
3214                    .downcast_ref::<LargeListArray>()
3215                    .expect("expected LargeListArray");
3216                let offsets = list.offsets().clone();
3217                let values = list.values().clone();
3218                let validity = list.nulls().cloned();
3219                Arc::new(LargeListArray::try_new(new_elem, offsets, values, validity).unwrap())
3220            } else {
3221                let list = col
3222                    .as_any()
3223                    .downcast_ref::<ListArray>()
3224                    .expect("expected ListArray");
3225                let offsets = list.offsets().clone();
3226                let values = list.values().clone();
3227                let validity = list.nulls().cloned();
3228                Arc::new(ListArray::try_new(new_elem, offsets, values, validity).unwrap())
3229            }
3230        }
3231        for (idx, f) in fields_json.iter().enumerate() {
3232            let name = f
3233                .get("name")
3234                .and_then(|n| n.as_str())
3235                .unwrap_or_else(|| panic!("field at index {idx} has no name"));
3236            let reader_schema = make_reader_schema_with_selected_fields_in_order(path, &[name]);
3237            let projected = read_alltypes_with_reader_schema(path, reader_schema);
3238            assert_eq!(
3239                projected.num_columns(),
3240                1,
3241                "projected batch should contain exactly the selected column '{name}'"
3242            );
3243            assert_eq!(
3244                projected.num_rows(),
3245                num_rows,
3246                "row count mismatch for projected column '{name}'"
3247            );
3248            let col_full = full.column(idx).clone();
3249            let full_field = schema_full.field(idx).as_ref().clone();
3250            let proj_field_ref = projected.schema().field(0).clone();
3251            let proj_field = proj_field_ref.as_ref();
3252            let top_meta = proj_field.metadata().clone();
3253            let (expected_field_ref, expected_col): (Arc<Field>, ArrayRef) =
3254                match (full_field.data_type(), proj_field.data_type()) {
3255                    (&DataType::List(_), DataType::List(proj_elem)) => {
3256                        let new_col =
3257                            rebuild_list_array_with_element(&col_full, proj_elem.clone(), false);
3258                        let nf = Field::new(
3259                            full_field.name().clone(),
3260                            proj_field.data_type().clone(),
3261                            full_field.is_nullable(),
3262                        )
3263                        .with_metadata(top_meta);
3264                        (Arc::new(nf), new_col)
3265                    }
3266                    (&DataType::LargeList(_), DataType::LargeList(proj_elem)) => {
3267                        let new_col =
3268                            rebuild_list_array_with_element(&col_full, proj_elem.clone(), true);
3269                        let nf = Field::new(
3270                            full_field.name().clone(),
3271                            proj_field.data_type().clone(),
3272                            full_field.is_nullable(),
3273                        )
3274                        .with_metadata(top_meta);
3275                        (Arc::new(nf), new_col)
3276                    }
3277                    _ => {
3278                        let nf = full_field.with_metadata(top_meta);
3279                        (Arc::new(nf), col_full)
3280                    }
3281                };
3282
3283            let expected = RecordBatch::try_new(
3284                Arc::new(Schema::new(vec![expected_field_ref])),
3285                vec![expected_col],
3286            )
3287            .unwrap();
3288            assert_eq!(
3289                projected, expected,
3290                "projected column '{name}' mismatch vs full read column"
3291            );
3292        }
3293    }
3294
3295    #[test]
3296    fn test_union_fields_avro_nullable_and_general_unions() {
3297        let path = "test/data/union_fields.avro";
3298        let batch = read_file(path, 1024, false);
3299        let schema = batch.schema();
3300        let idx = schema.index_of("nullable_int_nullfirst").unwrap();
3301        let a = batch.column(idx).as_primitive::<Int32Type>();
3302        assert_eq!(a.len(), 4);
3303        assert!(a.is_null(0));
3304        assert_eq!(a.value(1), 42);
3305        assert!(a.is_null(2));
3306        assert_eq!(a.value(3), 0);
3307        let idx = schema.index_of("nullable_string_nullsecond").unwrap();
3308        let s = batch
3309            .column(idx)
3310            .as_any()
3311            .downcast_ref::<StringArray>()
3312            .expect("nullable_string_nullsecond should be Utf8");
3313        assert_eq!(s.len(), 4);
3314        assert_eq!(s.value(0), "s1");
3315        assert!(s.is_null(1));
3316        assert_eq!(s.value(2), "s3");
3317        assert!(s.is_valid(3)); // empty string, not null
3318        assert_eq!(s.value(3), "");
3319        let idx = schema.index_of("union_prim").unwrap();
3320        let u = batch
3321            .column(idx)
3322            .as_any()
3323            .downcast_ref::<UnionArray>()
3324            .expect("union_prim should be Union");
3325        let fields = match u.data_type() {
3326            DataType::Union(fields, mode) => {
3327                assert!(matches!(mode, UnionMode::Dense), "expect dense unions");
3328                fields
3329            }
3330            other => panic!("expected Union, got {other:?}"),
3331        };
3332        let tid_by_name = |name: &str| -> i8 {
3333            for (tid, f) in fields.iter() {
3334                if f.name() == name {
3335                    return tid;
3336                }
3337            }
3338            panic!("union child '{name}' not found");
3339        };
3340        let expected_type_ids = vec![
3341            tid_by_name("long"),
3342            tid_by_name("int"),
3343            tid_by_name("float"),
3344            tid_by_name("double"),
3345        ];
3346        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3347        assert_eq!(
3348            type_ids, expected_type_ids,
3349            "branch selection for union_prim rows"
3350        );
3351        let longs = u
3352            .child(tid_by_name("long"))
3353            .as_any()
3354            .downcast_ref::<Int64Array>()
3355            .unwrap();
3356        assert_eq!(longs.len(), 1);
3357        let ints = u
3358            .child(tid_by_name("int"))
3359            .as_any()
3360            .downcast_ref::<Int32Array>()
3361            .unwrap();
3362        assert_eq!(ints.len(), 1);
3363        let floats = u
3364            .child(tid_by_name("float"))
3365            .as_any()
3366            .downcast_ref::<Float32Array>()
3367            .unwrap();
3368        assert_eq!(floats.len(), 1);
3369        let doubles = u
3370            .child(tid_by_name("double"))
3371            .as_any()
3372            .downcast_ref::<Float64Array>()
3373            .unwrap();
3374        assert_eq!(doubles.len(), 1);
3375        let idx = schema.index_of("union_bytes_vs_string").unwrap();
3376        let u = batch
3377            .column(idx)
3378            .as_any()
3379            .downcast_ref::<UnionArray>()
3380            .expect("union_bytes_vs_string should be Union");
3381        let fields = match u.data_type() {
3382            DataType::Union(fields, _) => fields,
3383            other => panic!("expected Union, got {other:?}"),
3384        };
3385        let tid_by_name = |name: &str| -> i8 {
3386            for (tid, f) in fields.iter() {
3387                if f.name() == name {
3388                    return tid;
3389                }
3390            }
3391            panic!("union child '{name}' not found");
3392        };
3393        let tid_bytes = tid_by_name("bytes");
3394        let tid_string = tid_by_name("string");
3395        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3396        assert_eq!(
3397            type_ids,
3398            vec![tid_bytes, tid_string, tid_string, tid_bytes],
3399            "branch selection for bytes/string union"
3400        );
3401        let s_child = u
3402            .child(tid_string)
3403            .as_any()
3404            .downcast_ref::<StringArray>()
3405            .unwrap();
3406        assert_eq!(s_child.len(), 2);
3407        assert_eq!(s_child.value(0), "hello");
3408        assert_eq!(s_child.value(1), "world");
3409        let b_child = u
3410            .child(tid_bytes)
3411            .as_any()
3412            .downcast_ref::<BinaryArray>()
3413            .unwrap();
3414        assert_eq!(b_child.len(), 2);
3415        assert_eq!(b_child.value(0), &[0x00, 0xFF, 0x7F]);
3416        assert_eq!(b_child.value(1), b""); // previously: &[]
3417        let idx = schema.index_of("union_enum_records_array_map").unwrap();
3418        let u = batch
3419            .column(idx)
3420            .as_any()
3421            .downcast_ref::<UnionArray>()
3422            .expect("union_enum_records_array_map should be Union");
3423        let fields = match u.data_type() {
3424            DataType::Union(fields, _) => fields,
3425            other => panic!("expected Union, got {other:?}"),
3426        };
3427        let mut tid_enum: Option<i8> = None;
3428        let mut tid_rec_a: Option<i8> = None;
3429        let mut tid_rec_b: Option<i8> = None;
3430        let mut tid_array: Option<i8> = None;
3431        for (tid, f) in fields.iter() {
3432            match f.data_type() {
3433                DataType::Dictionary(_, _) => tid_enum = Some(tid),
3434                DataType::Struct(childs) => {
3435                    if childs.len() == 2 && childs[0].name() == "a" && childs[1].name() == "b" {
3436                        tid_rec_a = Some(tid);
3437                    } else if childs.len() == 2
3438                        && childs[0].name() == "x"
3439                        && childs[1].name() == "y"
3440                    {
3441                        tid_rec_b = Some(tid);
3442                    }
3443                }
3444                DataType::List(_) => tid_array = Some(tid),
3445                _ => {}
3446            }
3447        }
3448        let (tid_enum, tid_rec_a, tid_rec_b, tid_array) = (
3449            tid_enum.expect("enum child"),
3450            tid_rec_a.expect("RecA child"),
3451            tid_rec_b.expect("RecB child"),
3452            tid_array.expect("array<long> child"),
3453        );
3454        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3455        assert_eq!(
3456            type_ids,
3457            vec![tid_enum, tid_rec_a, tid_rec_b, tid_array],
3458            "branch selection for complex union"
3459        );
3460        let dict = u
3461            .child(tid_enum)
3462            .as_any()
3463            .downcast_ref::<DictionaryArray<Int32Type>>()
3464            .unwrap();
3465        assert_eq!(dict.len(), 1);
3466        assert!(dict.is_valid(0));
3467        let rec_a = u
3468            .child(tid_rec_a)
3469            .as_any()
3470            .downcast_ref::<StructArray>()
3471            .unwrap();
3472        assert_eq!(rec_a.len(), 1);
3473        let a_val = rec_a
3474            .column_by_name("a")
3475            .unwrap()
3476            .as_any()
3477            .downcast_ref::<Int32Array>()
3478            .unwrap();
3479        assert_eq!(a_val.value(0), 7);
3480        let b_val = rec_a
3481            .column_by_name("b")
3482            .unwrap()
3483            .as_any()
3484            .downcast_ref::<StringArray>()
3485            .unwrap();
3486        assert_eq!(b_val.value(0), "x");
3487        // RecB row: {"x": 123456789, "y": b"\xFF\x00"}
3488        let rec_b = u
3489            .child(tid_rec_b)
3490            .as_any()
3491            .downcast_ref::<StructArray>()
3492            .unwrap();
3493        let x_val = rec_b
3494            .column_by_name("x")
3495            .unwrap()
3496            .as_any()
3497            .downcast_ref::<Int64Array>()
3498            .unwrap();
3499        assert_eq!(x_val.value(0), 123_456_789_i64);
3500        let y_val = rec_b
3501            .column_by_name("y")
3502            .unwrap()
3503            .as_any()
3504            .downcast_ref::<BinaryArray>()
3505            .unwrap();
3506        assert_eq!(y_val.value(0), &[0xFF, 0x00]);
3507        let arr = u
3508            .child(tid_array)
3509            .as_any()
3510            .downcast_ref::<ListArray>()
3511            .unwrap();
3512        assert_eq!(arr.len(), 1);
3513        let first_values = arr.value(0);
3514        let longs = first_values.as_any().downcast_ref::<Int64Array>().unwrap();
3515        assert_eq!(longs.len(), 3);
3516        assert_eq!(longs.value(0), 1);
3517        assert_eq!(longs.value(1), 2);
3518        assert_eq!(longs.value(2), 3);
3519        let idx = schema.index_of("union_date_or_fixed4").unwrap();
3520        let u = batch
3521            .column(idx)
3522            .as_any()
3523            .downcast_ref::<UnionArray>()
3524            .expect("union_date_or_fixed4 should be Union");
3525        let fields = match u.data_type() {
3526            DataType::Union(fields, _) => fields,
3527            other => panic!("expected Union, got {other:?}"),
3528        };
3529        let mut tid_date: Option<i8> = None;
3530        let mut tid_fixed: Option<i8> = None;
3531        for (tid, f) in fields.iter() {
3532            match f.data_type() {
3533                DataType::Date32 => tid_date = Some(tid),
3534                DataType::FixedSizeBinary(4) => tid_fixed = Some(tid),
3535                _ => {}
3536            }
3537        }
3538        let (tid_date, tid_fixed) = (tid_date.expect("date"), tid_fixed.expect("fixed(4)"));
3539        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3540        assert_eq!(
3541            type_ids,
3542            vec![tid_date, tid_fixed, tid_date, tid_fixed],
3543            "branch selection for date/fixed4 union"
3544        );
3545        let dates = u
3546            .child(tid_date)
3547            .as_any()
3548            .downcast_ref::<Date32Array>()
3549            .unwrap();
3550        assert_eq!(dates.len(), 2);
3551        assert_eq!(dates.value(0), 19_000); // ~2022‑01‑15
3552        assert_eq!(dates.value(1), 0); // epoch
3553        let fixed = u
3554            .child(tid_fixed)
3555            .as_any()
3556            .downcast_ref::<FixedSizeBinaryArray>()
3557            .unwrap();
3558        assert_eq!(fixed.len(), 2);
3559        assert_eq!(fixed.value(0), b"ABCD");
3560        assert_eq!(fixed.value(1), &[0x00, 0x11, 0x22, 0x33]);
3561    }
3562
3563    #[test]
3564    fn test_union_schema_resolution_all_type_combinations() {
3565        let path = "test/data/union_fields.avro";
3566        let baseline = read_file(path, 1024, false);
3567        let baseline_schema = baseline.schema();
3568        let mut root = load_writer_schema_json(path);
3569        assert_eq!(root["type"], "record", "writer schema must be a record");
3570        let fields = root
3571            .get_mut("fields")
3572            .and_then(|f| f.as_array_mut())
3573            .expect("record has fields");
3574        fn is_named_type(obj: &Value, ty: &str, nm: &str) -> bool {
3575            obj.get("type").and_then(|v| v.as_str()) == Some(ty)
3576                && obj.get("name").and_then(|v| v.as_str()) == Some(nm)
3577        }
3578        fn is_logical(obj: &Value, prim: &str, lt: &str) -> bool {
3579            obj.get("type").and_then(|v| v.as_str()) == Some(prim)
3580                && obj.get("logicalType").and_then(|v| v.as_str()) == Some(lt)
3581        }
3582        fn find_first(arr: &[Value], pred: impl Fn(&Value) -> bool) -> Option<Value> {
3583            arr.iter().find(|v| pred(v)).cloned()
3584        }
3585        fn prim(s: &str) -> Value {
3586            Value::String(s.to_string())
3587        }
3588        for f in fields.iter_mut() {
3589            let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
3590                continue;
3591            };
3592            match name {
3593                // Flip null ordering – should not affect values
3594                "nullable_int_nullfirst" => {
3595                    f["type"] = json!(["int", "null"]);
3596                }
3597                "nullable_string_nullsecond" => {
3598                    f["type"] = json!(["null", "string"]);
3599                }
3600                "union_prim" => {
3601                    let orig = f["type"].as_array().unwrap().clone();
3602                    let long = prim("long");
3603                    let double = prim("double");
3604                    let string = prim("string");
3605                    let bytes = prim("bytes");
3606                    let boolean = prim("boolean");
3607                    assert!(orig.contains(&long));
3608                    assert!(orig.contains(&double));
3609                    assert!(orig.contains(&string));
3610                    assert!(orig.contains(&bytes));
3611                    assert!(orig.contains(&boolean));
3612                    f["type"] = json!([long, double, string, bytes, boolean]);
3613                }
3614                "union_bytes_vs_string" => {
3615                    f["type"] = json!(["string", "bytes"]);
3616                }
3617                "union_fixed_dur_decfix" => {
3618                    let orig = f["type"].as_array().unwrap().clone();
3619                    let fx8 = find_first(&orig, |o| is_named_type(o, "fixed", "Fx8")).unwrap();
3620                    let dur12 = find_first(&orig, |o| is_named_type(o, "fixed", "Dur12")).unwrap();
3621                    let decfix16 =
3622                        find_first(&orig, |o| is_named_type(o, "fixed", "DecFix16")).unwrap();
3623                    f["type"] = json!([decfix16, dur12, fx8]);
3624                }
3625                "union_enum_records_array_map" => {
3626                    let orig = f["type"].as_array().unwrap().clone();
3627                    let enum_color = find_first(&orig, |o| {
3628                        o.get("type").and_then(|v| v.as_str()) == Some("enum")
3629                    })
3630                    .unwrap();
3631                    let rec_a = find_first(&orig, |o| is_named_type(o, "record", "RecA")).unwrap();
3632                    let rec_b = find_first(&orig, |o| is_named_type(o, "record", "RecB")).unwrap();
3633                    let arr = find_first(&orig, |o| {
3634                        o.get("type").and_then(|v| v.as_str()) == Some("array")
3635                    })
3636                    .unwrap();
3637                    let map = find_first(&orig, |o| {
3638                        o.get("type").and_then(|v| v.as_str()) == Some("map")
3639                    })
3640                    .unwrap();
3641                    f["type"] = json!([arr, map, rec_b, rec_a, enum_color]);
3642                }
3643                "union_date_or_fixed4" => {
3644                    let orig = f["type"].as_array().unwrap().clone();
3645                    let date = find_first(&orig, |o| is_logical(o, "int", "date")).unwrap();
3646                    let fx4 = find_first(&orig, |o| is_named_type(o, "fixed", "Fx4")).unwrap();
3647                    f["type"] = json!([fx4, date]);
3648                }
3649                "union_time_millis_or_enum" => {
3650                    let orig = f["type"].as_array().unwrap().clone();
3651                    let time_ms =
3652                        find_first(&orig, |o| is_logical(o, "int", "time-millis")).unwrap();
3653                    let en = find_first(&orig, |o| {
3654                        o.get("type").and_then(|v| v.as_str()) == Some("enum")
3655                    })
3656                    .unwrap();
3657                    f["type"] = json!([en, time_ms]);
3658                }
3659                "union_time_micros_or_string" => {
3660                    let orig = f["type"].as_array().unwrap().clone();
3661                    let time_us =
3662                        find_first(&orig, |o| is_logical(o, "long", "time-micros")).unwrap();
3663                    f["type"] = json!(["string", time_us]);
3664                }
3665                "union_ts_millis_utc_or_array" => {
3666                    let orig = f["type"].as_array().unwrap().clone();
3667                    let ts_ms =
3668                        find_first(&orig, |o| is_logical(o, "long", "timestamp-millis")).unwrap();
3669                    let arr = find_first(&orig, |o| {
3670                        o.get("type").and_then(|v| v.as_str()) == Some("array")
3671                    })
3672                    .unwrap();
3673                    f["type"] = json!([arr, ts_ms]);
3674                }
3675                "union_ts_micros_local_or_bytes" => {
3676                    let orig = f["type"].as_array().unwrap().clone();
3677                    let lts_us =
3678                        find_first(&orig, |o| is_logical(o, "long", "local-timestamp-micros"))
3679                            .unwrap();
3680                    f["type"] = json!(["bytes", lts_us]);
3681                }
3682                "union_uuid_or_fixed10" => {
3683                    let orig = f["type"].as_array().unwrap().clone();
3684                    let uuid = find_first(&orig, |o| is_logical(o, "string", "uuid")).unwrap();
3685                    let fx10 = find_first(&orig, |o| is_named_type(o, "fixed", "Fx10")).unwrap();
3686                    f["type"] = json!([fx10, uuid]);
3687                }
3688                "union_dec_bytes_or_dec_fixed" => {
3689                    let orig = f["type"].as_array().unwrap().clone();
3690                    let dec_bytes = find_first(&orig, |o| {
3691                        o.get("type").and_then(|v| v.as_str()) == Some("bytes")
3692                            && o.get("logicalType").and_then(|v| v.as_str()) == Some("decimal")
3693                    })
3694                    .unwrap();
3695                    let dec_fix = find_first(&orig, |o| {
3696                        is_named_type(o, "fixed", "DecFix20")
3697                            && o.get("logicalType").and_then(|v| v.as_str()) == Some("decimal")
3698                    })
3699                    .unwrap();
3700                    f["type"] = json!([dec_fix, dec_bytes]);
3701                }
3702                "union_null_bytes_string" => {
3703                    f["type"] = json!(["bytes", "string", "null"]);
3704                }
3705                "array_of_union" => {
3706                    let obj = f
3707                        .get_mut("type")
3708                        .expect("array type")
3709                        .as_object_mut()
3710                        .unwrap();
3711                    obj.insert("items".to_string(), json!(["string", "long"]));
3712                }
3713                "map_of_union" => {
3714                    let obj = f
3715                        .get_mut("type")
3716                        .expect("map type")
3717                        .as_object_mut()
3718                        .unwrap();
3719                    obj.insert("values".to_string(), json!(["double", "null"]));
3720                }
3721                "record_with_union_field" => {
3722                    let rec = f
3723                        .get_mut("type")
3724                        .expect("record type")
3725                        .as_object_mut()
3726                        .unwrap();
3727                    let rec_fields = rec.get_mut("fields").unwrap().as_array_mut().unwrap();
3728                    let mut found = false;
3729                    for rf in rec_fields.iter_mut() {
3730                        if rf.get("name").and_then(|v| v.as_str()) == Some("u") {
3731                            rf["type"] = json!(["string", "long"]); // rely on int→long promotion
3732                            found = true;
3733                            break;
3734                        }
3735                    }
3736                    assert!(found, "field 'u' expected in HasUnion");
3737                }
3738                "union_ts_micros_utc_or_map" => {
3739                    let orig = f["type"].as_array().unwrap().clone();
3740                    let ts_us =
3741                        find_first(&orig, |o| is_logical(o, "long", "timestamp-micros")).unwrap();
3742                    let map = find_first(&orig, |o| {
3743                        o.get("type").and_then(|v| v.as_str()) == Some("map")
3744                    })
3745                    .unwrap();
3746                    f["type"] = json!([map, ts_us]);
3747                }
3748                "union_ts_millis_local_or_string" => {
3749                    let orig = f["type"].as_array().unwrap().clone();
3750                    let lts_ms =
3751                        find_first(&orig, |o| is_logical(o, "long", "local-timestamp-millis"))
3752                            .unwrap();
3753                    f["type"] = json!(["string", lts_ms]);
3754                }
3755                "union_bool_or_string" => {
3756                    f["type"] = json!(["string", "boolean"]);
3757                }
3758                _ => {}
3759            }
3760        }
3761        let reader_schema = AvroSchema::new(root.to_string());
3762        let resolved = read_alltypes_with_reader_schema(path, reader_schema);
3763
3764        fn branch_token(dt: &DataType) -> String {
3765            match dt {
3766                DataType::Null => "null".into(),
3767                DataType::Boolean => "boolean".into(),
3768                DataType::Int32 => "int".into(),
3769                DataType::Int64 => "long".into(),
3770                DataType::Float32 => "float".into(),
3771                DataType::Float64 => "double".into(),
3772                DataType::Binary => "bytes".into(),
3773                DataType::Utf8 => "string".into(),
3774                DataType::Date32 => "date".into(),
3775                DataType::Time32(arrow_schema::TimeUnit::Millisecond) => "time-millis".into(),
3776                DataType::Time64(arrow_schema::TimeUnit::Microsecond) => "time-micros".into(),
3777                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => if tz.is_some() {
3778                    "timestamp-millis"
3779                } else {
3780                    "local-timestamp-millis"
3781                }
3782                .into(),
3783                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => if tz.is_some() {
3784                    "timestamp-micros"
3785                } else {
3786                    "local-timestamp-micros"
3787                }
3788                .into(),
3789                DataType::Interval(IntervalUnit::MonthDayNano) => "duration".into(),
3790                DataType::FixedSizeBinary(n) => format!("fixed{n}"),
3791                DataType::Dictionary(_, _) => "enum".into(),
3792                DataType::Decimal128(p, s) => format!("decimal({p},{s})"),
3793                DataType::Decimal256(p, s) => format!("decimal({p},{s})"),
3794                #[cfg(feature = "small_decimals")]
3795                DataType::Decimal64(p, s) => format!("decimal({p},{s})"),
3796                DataType::Struct(fields) => {
3797                    if fields.len() == 2 && fields[0].name() == "a" && fields[1].name() == "b" {
3798                        "record:RecA".into()
3799                    } else if fields.len() == 2
3800                        && fields[0].name() == "x"
3801                        && fields[1].name() == "y"
3802                    {
3803                        "record:RecB".into()
3804                    } else {
3805                        "record".into()
3806                    }
3807                }
3808                DataType::List(_) => "array".into(),
3809                DataType::Map(_, _) => "map".into(),
3810                other => format!("{other:?}"),
3811            }
3812        }
3813
3814        fn union_tokens(u: &UnionArray) -> (Vec<i8>, HashMap<i8, String>) {
3815            let fields = match u.data_type() {
3816                DataType::Union(fields, _) => fields,
3817                other => panic!("expected Union, got {other:?}"),
3818            };
3819            let mut dict: HashMap<i8, String> = HashMap::with_capacity(fields.len());
3820            for (tid, f) in fields.iter() {
3821                dict.insert(tid, branch_token(f.data_type()));
3822            }
3823            let ids: Vec<i8> = u.type_ids().iter().copied().collect();
3824            (ids, dict)
3825        }
3826
3827        fn expected_token(field_name: &str, writer_token: &str) -> String {
3828            match field_name {
3829                "union_prim" => match writer_token {
3830                    "int" => "long".into(),
3831                    "float" => "double".into(),
3832                    other => other.into(),
3833                },
3834                "record_with_union_field.u" => match writer_token {
3835                    "int" => "long".into(),
3836                    other => other.into(),
3837                },
3838                _ => writer_token.into(),
3839            }
3840        }
3841
3842        fn get_union<'a>(
3843            rb: &'a RecordBatch,
3844            schema: arrow_schema::SchemaRef,
3845            fname: &str,
3846        ) -> &'a UnionArray {
3847            let idx = schema.index_of(fname).unwrap();
3848            rb.column(idx)
3849                .as_any()
3850                .downcast_ref::<UnionArray>()
3851                .unwrap_or_else(|| panic!("{fname} should be a Union"))
3852        }
3853
3854        fn assert_union_equivalent(field_name: &str, u_writer: &UnionArray, u_reader: &UnionArray) {
3855            let (ids_w, dict_w) = union_tokens(u_writer);
3856            let (ids_r, dict_r) = union_tokens(u_reader);
3857            assert_eq!(
3858                ids_w.len(),
3859                ids_r.len(),
3860                "{field_name}: row count mismatch between baseline and resolved"
3861            );
3862            for (i, (id_w, id_r)) in ids_w.iter().zip(ids_r.iter()).enumerate() {
3863                let w_tok = dict_w.get(id_w).unwrap();
3864                let want = expected_token(field_name, w_tok);
3865                let got = dict_r.get(id_r).unwrap();
3866                assert_eq!(
3867                    got, &want,
3868                    "{field_name}: row {i} resolved to wrong union branch (writer={w_tok}, expected={want}, got={got})"
3869                );
3870            }
3871        }
3872
3873        for (fname, dt) in [
3874            ("nullable_int_nullfirst", DataType::Int32),
3875            ("nullable_string_nullsecond", DataType::Utf8),
3876        ] {
3877            let idx_b = baseline_schema.index_of(fname).unwrap();
3878            let idx_r = resolved.schema().index_of(fname).unwrap();
3879            let col_b = baseline.column(idx_b);
3880            let col_r = resolved.column(idx_r);
3881            assert_eq!(
3882                col_b.data_type(),
3883                &dt,
3884                "baseline {fname} should decode as non-union with nullability"
3885            );
3886            assert_eq!(
3887                col_b.as_ref(),
3888                col_r.as_ref(),
3889                "{fname}: values must be identical regardless of null-branch order"
3890            );
3891        }
3892        let union_fields = [
3893            "union_prim",
3894            "union_bytes_vs_string",
3895            "union_fixed_dur_decfix",
3896            "union_enum_records_array_map",
3897            "union_date_or_fixed4",
3898            "union_time_millis_or_enum",
3899            "union_time_micros_or_string",
3900            "union_ts_millis_utc_or_array",
3901            "union_ts_micros_local_or_bytes",
3902            "union_uuid_or_fixed10",
3903            "union_dec_bytes_or_dec_fixed",
3904            "union_null_bytes_string",
3905            "union_ts_micros_utc_or_map",
3906            "union_ts_millis_local_or_string",
3907            "union_bool_or_string",
3908        ];
3909        for fname in union_fields {
3910            let u_b = get_union(&baseline, baseline_schema.clone(), fname);
3911            let u_r = get_union(&resolved, resolved.schema(), fname);
3912            assert_union_equivalent(fname, u_b, u_r);
3913        }
3914        {
3915            let fname = "array_of_union";
3916            let idx_b = baseline_schema.index_of(fname).unwrap();
3917            let idx_r = resolved.schema().index_of(fname).unwrap();
3918            let arr_b = baseline
3919                .column(idx_b)
3920                .as_any()
3921                .downcast_ref::<ListArray>()
3922                .expect("array_of_union should be a List");
3923            let arr_r = resolved
3924                .column(idx_r)
3925                .as_any()
3926                .downcast_ref::<ListArray>()
3927                .expect("array_of_union should be a List");
3928            assert_eq!(
3929                arr_b.value_offsets(),
3930                arr_r.value_offsets(),
3931                "{fname}: list offsets changed after resolution"
3932            );
3933            let u_b = arr_b
3934                .values()
3935                .as_any()
3936                .downcast_ref::<UnionArray>()
3937                .expect("array items should be Union");
3938            let u_r = arr_r
3939                .values()
3940                .as_any()
3941                .downcast_ref::<UnionArray>()
3942                .expect("array items should be Union");
3943            let (ids_b, dict_b) = union_tokens(u_b);
3944            let (ids_r, dict_r) = union_tokens(u_r);
3945            assert_eq!(ids_b.len(), ids_r.len(), "{fname}: values length mismatch");
3946            for (i, (id_b, id_r)) in ids_b.iter().zip(ids_r.iter()).enumerate() {
3947                let w_tok = dict_b.get(id_b).unwrap();
3948                let got = dict_r.get(id_r).unwrap();
3949                assert_eq!(
3950                    got, w_tok,
3951                    "{fname}: value {i} resolved to wrong branch (writer={w_tok}, got={got})"
3952                );
3953            }
3954        }
3955        {
3956            let fname = "map_of_union";
3957            let idx_b = baseline_schema.index_of(fname).unwrap();
3958            let idx_r = resolved.schema().index_of(fname).unwrap();
3959            let map_b = baseline
3960                .column(idx_b)
3961                .as_any()
3962                .downcast_ref::<MapArray>()
3963                .expect("map_of_union should be a Map");
3964            let map_r = resolved
3965                .column(idx_r)
3966                .as_any()
3967                .downcast_ref::<MapArray>()
3968                .expect("map_of_union should be a Map");
3969            assert_eq!(
3970                map_b.value_offsets(),
3971                map_r.value_offsets(),
3972                "{fname}: map value offsets changed after resolution"
3973            );
3974            let ent_b = map_b.entries();
3975            let ent_r = map_r.entries();
3976            let val_b_any = ent_b.column(1).as_ref();
3977            let val_r_any = ent_r.column(1).as_ref();
3978            let b_union = val_b_any.as_any().downcast_ref::<UnionArray>();
3979            let r_union = val_r_any.as_any().downcast_ref::<UnionArray>();
3980            if let (Some(u_b), Some(u_r)) = (b_union, r_union) {
3981                assert_union_equivalent(fname, u_b, u_r);
3982            } else {
3983                assert_eq!(
3984                    val_b_any.data_type(),
3985                    val_r_any.data_type(),
3986                    "{fname}: value data types differ after resolution"
3987                );
3988                assert_eq!(
3989                    val_b_any, val_r_any,
3990                    "{fname}: value arrays differ after resolution (nullable value column case)"
3991                );
3992                let value_nullable = |m: &MapArray| -> bool {
3993                    match m.data_type() {
3994                        DataType::Map(entries_field, _sorted) => match entries_field.data_type() {
3995                            DataType::Struct(fields) => {
3996                                assert_eq!(fields.len(), 2, "entries struct must have 2 fields");
3997                                assert_eq!(fields[0].name(), "key");
3998                                assert_eq!(fields[1].name(), "value");
3999                                fields[1].is_nullable()
4000                            }
4001                            other => panic!("Map entries field must be Struct, got {other:?}"),
4002                        },
4003                        other => panic!("expected Map data type, got {other:?}"),
4004                    }
4005                };
4006                assert!(
4007                    value_nullable(map_b),
4008                    "{fname}: baseline Map value field should be nullable per Arrow spec"
4009                );
4010                assert!(
4011                    value_nullable(map_r),
4012                    "{fname}: resolved Map value field should be nullable per Arrow spec"
4013                );
4014            }
4015        }
4016        {
4017            let fname = "record_with_union_field";
4018            let idx_b = baseline_schema.index_of(fname).unwrap();
4019            let idx_r = resolved.schema().index_of(fname).unwrap();
4020            let rec_b = baseline
4021                .column(idx_b)
4022                .as_any()
4023                .downcast_ref::<StructArray>()
4024                .expect("record_with_union_field should be a Struct");
4025            let rec_r = resolved
4026                .column(idx_r)
4027                .as_any()
4028                .downcast_ref::<StructArray>()
4029                .expect("record_with_union_field should be a Struct");
4030            let u_b = rec_b
4031                .column_by_name("u")
4032                .unwrap()
4033                .as_any()
4034                .downcast_ref::<UnionArray>()
4035                .expect("field 'u' should be Union (baseline)");
4036            let u_r = rec_r
4037                .column_by_name("u")
4038                .unwrap()
4039                .as_any()
4040                .downcast_ref::<UnionArray>()
4041                .expect("field 'u' should be Union (resolved)");
4042            assert_union_equivalent("record_with_union_field.u", u_b, u_r);
4043        }
4044    }
4045
4046    #[test]
4047    fn test_union_fields_end_to_end_expected_arrays() {
4048        fn tid_by_name(fields: &UnionFields, want: &str) -> i8 {
4049            for (tid, f) in fields.iter() {
4050                if f.name() == want {
4051                    return tid;
4052                }
4053            }
4054            panic!("union child '{want}' not found")
4055        }
4056
4057        fn tid_by_dt(fields: &UnionFields, pred: impl Fn(&DataType) -> bool) -> i8 {
4058            for (tid, f) in fields.iter() {
4059                if pred(f.data_type()) {
4060                    return tid;
4061                }
4062            }
4063            panic!("no union child matches predicate");
4064        }
4065
4066        fn uuid16_from_str(s: &str) -> [u8; 16] {
4067            fn hex(b: u8) -> u8 {
4068                match b {
4069                    b'0'..=b'9' => b - b'0',
4070                    b'a'..=b'f' => b - b'a' + 10,
4071                    b'A'..=b'F' => b - b'A' + 10,
4072                    _ => panic!("invalid hex"),
4073                }
4074            }
4075            let mut out = [0u8; 16];
4076            let bytes = s.as_bytes();
4077            let (mut i, mut j) = (0, 0);
4078            while i < bytes.len() {
4079                if bytes[i] == b'-' {
4080                    i += 1;
4081                    continue;
4082                }
4083                let hi = hex(bytes[i]);
4084                let lo = hex(bytes[i + 1]);
4085                out[j] = (hi << 4) | lo;
4086                j += 1;
4087                i += 2;
4088            }
4089            assert_eq!(j, 16, "uuid must decode to 16 bytes");
4090            out
4091        }
4092
4093        fn empty_child_for(dt: &DataType) -> Arc<dyn Array> {
4094            match dt {
4095                DataType::Null => Arc::new(NullArray::new(0)),
4096                DataType::Boolean => Arc::new(BooleanArray::from(Vec::<bool>::new())),
4097                DataType::Int32 => Arc::new(Int32Array::from(Vec::<i32>::new())),
4098                DataType::Int64 => Arc::new(Int64Array::from(Vec::<i64>::new())),
4099                DataType::Float32 => Arc::new(arrow_array::Float32Array::from(Vec::<f32>::new())),
4100                DataType::Float64 => Arc::new(arrow_array::Float64Array::from(Vec::<f64>::new())),
4101                DataType::Binary => Arc::new(BinaryArray::from(Vec::<&[u8]>::new())),
4102                DataType::Utf8 => Arc::new(StringArray::from(Vec::<&str>::new())),
4103                DataType::Date32 => Arc::new(arrow_array::Date32Array::from(Vec::<i32>::new())),
4104                DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
4105                    Arc::new(Time32MillisecondArray::from(Vec::<i32>::new()))
4106                }
4107                DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
4108                    Arc::new(Time64MicrosecondArray::from(Vec::<i64>::new()))
4109                }
4110                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
4111                    let a = TimestampMillisecondArray::from(Vec::<i64>::new());
4112                    Arc::new(if let Some(tz) = tz {
4113                        a.with_timezone(tz.clone())
4114                    } else {
4115                        a
4116                    })
4117                }
4118                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
4119                    let a = TimestampMicrosecondArray::from(Vec::<i64>::new());
4120                    Arc::new(if let Some(tz) = tz {
4121                        a.with_timezone(tz.clone())
4122                    } else {
4123                        a
4124                    })
4125                }
4126                DataType::Interval(IntervalUnit::MonthDayNano) => {
4127                    Arc::new(arrow_array::IntervalMonthDayNanoArray::from(Vec::<
4128                        IntervalMonthDayNano,
4129                    >::new(
4130                    )))
4131                }
4132                DataType::FixedSizeBinary(n) => Arc::new(FixedSizeBinaryArray::new_null(*n, 0)),
4133                DataType::Dictionary(k, v) => {
4134                    assert_eq!(**k, DataType::Int32, "expect int32 keys for enums");
4135                    let keys = Int32Array::from(Vec::<i32>::new());
4136                    let values = match v.as_ref() {
4137                        DataType::Utf8 => {
4138                            Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
4139                        }
4140                        other => panic!("unexpected dictionary value type {other:?}"),
4141                    };
4142                    Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
4143                }
4144                DataType::List(field) => {
4145                    let values: ArrayRef = match field.data_type() {
4146                        DataType::Int32 => {
4147                            Arc::new(Int32Array::from(Vec::<i32>::new())) as ArrayRef
4148                        }
4149                        DataType::Int64 => {
4150                            Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
4151                        }
4152                        DataType::Utf8 => {
4153                            Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
4154                        }
4155                        DataType::Union(_, _) => {
4156                            let (uf, _) = if let DataType::Union(f, m) = field.data_type() {
4157                                (f.clone(), m)
4158                            } else {
4159                                unreachable!()
4160                            };
4161                            let children: Vec<ArrayRef> = uf
4162                                .iter()
4163                                .map(|(_, f)| empty_child_for(f.data_type()))
4164                                .collect();
4165                            Arc::new(
4166                                UnionArray::try_new(
4167                                    uf.clone(),
4168                                    ScalarBuffer::<i8>::from(Vec::<i8>::new()),
4169                                    Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
4170                                    children,
4171                                )
4172                                .unwrap(),
4173                            ) as ArrayRef
4174                        }
4175                        other => panic!("unsupported list item type: {other:?}"),
4176                    };
4177                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
4178                    Arc::new(ListArray::try_new(field.clone(), offsets, values, None).unwrap())
4179                }
4180                DataType::Map(entry_field, ordered) => {
4181                    let DataType::Struct(childs) = entry_field.data_type() else {
4182                        panic!("map entries must be struct")
4183                    };
4184                    let key_field = &childs[0];
4185                    let val_field = &childs[1];
4186                    assert_eq!(key_field.data_type(), &DataType::Utf8);
4187                    let keys = StringArray::from(Vec::<&str>::new());
4188                    let vals: ArrayRef = match val_field.data_type() {
4189                        DataType::Float64 => {
4190                            Arc::new(arrow_array::Float64Array::from(Vec::<f64>::new())) as ArrayRef
4191                        }
4192                        DataType::Int64 => {
4193                            Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
4194                        }
4195                        DataType::Utf8 => {
4196                            Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
4197                        }
4198                        DataType::Union(uf, _) => {
4199                            let ch: Vec<ArrayRef> = uf
4200                                .iter()
4201                                .map(|(_, f)| empty_child_for(f.data_type()))
4202                                .collect();
4203                            Arc::new(
4204                                UnionArray::try_new(
4205                                    uf.clone(),
4206                                    ScalarBuffer::<i8>::from(Vec::<i8>::new()),
4207                                    Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
4208                                    ch,
4209                                )
4210                                .unwrap(),
4211                            ) as ArrayRef
4212                        }
4213                        other => panic!("unsupported map value type: {other:?}"),
4214                    };
4215                    let entries = StructArray::new(
4216                        Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
4217                        vec![Arc::new(keys) as ArrayRef, vals],
4218                        None,
4219                    );
4220                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
4221                    Arc::new(MapArray::new(
4222                        entry_field.clone(),
4223                        offsets,
4224                        entries,
4225                        None,
4226                        *ordered,
4227                    ))
4228                }
4229                other => panic!("empty_child_for: unhandled type {other:?}"),
4230            }
4231        }
4232
4233        fn mk_dense_union(
4234            fields: &UnionFields,
4235            type_ids: Vec<i8>,
4236            offsets: Vec<i32>,
4237            provide: impl Fn(&Field) -> Option<ArrayRef>,
4238        ) -> ArrayRef {
4239            let children: Vec<ArrayRef> = fields
4240                .iter()
4241                .map(|(_, f)| provide(f).unwrap_or_else(|| empty_child_for(f.data_type())))
4242                .collect();
4243
4244            Arc::new(
4245                UnionArray::try_new(
4246                    fields.clone(),
4247                    ScalarBuffer::<i8>::from(type_ids),
4248                    Some(ScalarBuffer::<i32>::from(offsets)),
4249                    children,
4250                )
4251                .unwrap(),
4252            ) as ArrayRef
4253        }
4254
4255        // Dates / times / timestamps from the Avro content block:
4256        let date_a: i32 = 19_000;
4257        let time_ms_a: i32 = 13 * 3_600_000 + 45 * 60_000 + 30_000 + 123;
4258        let time_us_b: i64 = 23 * 3_600_000_000 + 59 * 60_000_000 + 59 * 1_000_000 + 999_999;
4259        let ts_ms_2024_01_01: i64 = 1_704_067_200_000;
4260        let ts_us_2024_01_01: i64 = ts_ms_2024_01_01 * 1000;
4261        // Fixed / bytes-like values:
4262        let fx8_a: [u8; 8] = *b"ABCDEFGH";
4263        let fx4_abcd: [u8; 4] = *b"ABCD";
4264        let fx4_misc: [u8; 4] = [0x00, 0x11, 0x22, 0x33];
4265        let fx10_ascii: [u8; 10] = *b"0123456789";
4266        let fx10_aa: [u8; 10] = [0xAA; 10];
4267        // Duration logical values as MonthDayNano:
4268        let dur_a = IntervalMonthDayNanoType::make_value(1, 2, 3_000_000_000);
4269        let dur_b = IntervalMonthDayNanoType::make_value(12, 31, 999_000_000);
4270        // UUID logical values (stored as 16-byte FixedSizeBinary in Arrow):
4271        let uuid1 = uuid16_from_str("fe7bc30b-4ce8-4c5e-b67c-2234a2d38e66");
4272        let uuid2 = uuid16_from_str("0826cc06-d2e3-4599-b4ad-af5fa6905cdb");
4273        // Decimals from Avro content:
4274        let dec_b_scale2_pos: i128 = 123_456; // "1234.56" bytes-decimal -> (precision=10, scale=2)
4275        let dec_fix16_neg: i128 = -101; // "-1.01" fixed(16) decimal(10,2)
4276        let dec_fix20_s4: i128 = 1_234_567_891_234; // "123456789.1234" fixed(20) decimal(20,4)
4277        let dec_fix20_s4_neg: i128 = -123; // "-0.0123" fixed(20) decimal(20,4)
4278        let path = "test/data/union_fields.avro";
4279        let actual = read_file(path, 1024, false);
4280        let schema = actual.schema();
4281        // Helper to fetch union metadata for a column
4282        let get_union = |name: &str| -> (UnionFields, UnionMode) {
4283            let idx = schema.index_of(name).unwrap();
4284            match schema.field(idx).data_type() {
4285                DataType::Union(f, m) => (f.clone(), *m),
4286                other => panic!("{name} should be a Union, got {other:?}"),
4287            }
4288        };
4289        let mut expected_cols: Vec<ArrayRef> = Vec::with_capacity(schema.fields().len());
4290        // 1) ["null","int"]: Int32 (nullable)
4291        expected_cols.push(Arc::new(Int32Array::from(vec![
4292            None,
4293            Some(42),
4294            None,
4295            Some(0),
4296        ])));
4297        // 2) ["string","null"]: Utf8 (nullable)
4298        expected_cols.push(Arc::new(StringArray::from(vec![
4299            Some("s1"),
4300            None,
4301            Some("s3"),
4302            Some(""),
4303        ])));
4304        // 3) union_prim: ["boolean","int","long","float","double","bytes","string"]
4305        {
4306            let (uf, mode) = get_union("union_prim");
4307            assert!(matches!(mode, UnionMode::Dense));
4308            let generated_names: Vec<&str> = uf.iter().map(|(_, f)| f.name().as_str()).collect();
4309            let expected_names = vec![
4310                "boolean", "int", "long", "float", "double", "bytes", "string",
4311            ];
4312            assert_eq!(
4313                generated_names, expected_names,
4314                "Field names for union_prim are incorrect"
4315            );
4316            let tids = vec![
4317                tid_by_name(&uf, "long"),
4318                tid_by_name(&uf, "int"),
4319                tid_by_name(&uf, "float"),
4320                tid_by_name(&uf, "double"),
4321            ];
4322            let offs = vec![0, 0, 0, 0];
4323            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4324                "int" => Some(Arc::new(Int32Array::from(vec![-1])) as ArrayRef),
4325                "long" => Some(Arc::new(Int64Array::from(vec![1_234_567_890_123i64])) as ArrayRef),
4326                "float" => {
4327                    Some(Arc::new(arrow_array::Float32Array::from(vec![1.25f32])) as ArrayRef)
4328                }
4329                "double" => {
4330                    Some(Arc::new(arrow_array::Float64Array::from(vec![-2.5f64])) as ArrayRef)
4331                }
4332                _ => None,
4333            });
4334            expected_cols.push(arr);
4335        }
4336        // 4) union_bytes_vs_string: ["bytes","string"]
4337        {
4338            let (uf, _) = get_union("union_bytes_vs_string");
4339            let tids = vec![
4340                tid_by_name(&uf, "bytes"),
4341                tid_by_name(&uf, "string"),
4342                tid_by_name(&uf, "string"),
4343                tid_by_name(&uf, "bytes"),
4344            ];
4345            let offs = vec![0, 0, 1, 1];
4346            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4347                "bytes" => Some(
4348                    Arc::new(BinaryArray::from(vec![&[0x00, 0xFF, 0x7F][..], &[][..]])) as ArrayRef,
4349                ),
4350                "string" => Some(Arc::new(StringArray::from(vec!["hello", "world"])) as ArrayRef),
4351                _ => None,
4352            });
4353            expected_cols.push(arr);
4354        }
4355        // 5) union_fixed_dur_decfix: [Fx8, Dur12, DecFix16(decimal(10,2))]
4356        {
4357            let (uf, _) = get_union("union_fixed_dur_decfix");
4358            let tid_fx8 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(8)));
4359            let tid_dur = tid_by_dt(&uf, |dt| {
4360                matches!(
4361                    dt,
4362                    DataType::Interval(arrow_schema::IntervalUnit::MonthDayNano)
4363                )
4364            });
4365            let tid_dec = tid_by_dt(&uf, |dt| match dt {
4366                #[cfg(feature = "small_decimals")]
4367                DataType::Decimal64(10, 2) => true,
4368                DataType::Decimal128(10, 2) | DataType::Decimal256(10, 2) => true,
4369                _ => false,
4370            });
4371            let tids = vec![tid_fx8, tid_dur, tid_dec, tid_dur];
4372            let offs = vec![0, 0, 0, 1];
4373            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4374                DataType::FixedSizeBinary(8) => {
4375                    let it = [Some(fx8_a)].into_iter();
4376                    Some(Arc::new(
4377                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 8).unwrap(),
4378                    ) as ArrayRef)
4379                }
4380                DataType::Interval(IntervalUnit::MonthDayNano) => {
4381                    Some(Arc::new(arrow_array::IntervalMonthDayNanoArray::from(vec![
4382                        dur_a, dur_b,
4383                    ])) as ArrayRef)
4384                }
4385                #[cfg(feature = "small_decimals")]
4386                DataType::Decimal64(10, 2) => {
4387                    let a = arrow_array::Decimal64Array::from_iter_values([dec_fix16_neg as i64]);
4388                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4389                }
4390                DataType::Decimal128(10, 2) => {
4391                    let a = arrow_array::Decimal128Array::from_iter_values([dec_fix16_neg]);
4392                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4393                }
4394                DataType::Decimal256(10, 2) => {
4395                    let a = arrow_array::Decimal256Array::from_iter_values([i256::from_i128(
4396                        dec_fix16_neg,
4397                    )]);
4398                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4399                }
4400                _ => None,
4401            });
4402            let generated_names: Vec<&str> = uf.iter().map(|(_, f)| f.name().as_str()).collect();
4403            let expected_names = vec!["Fx8", "Dur12", "DecFix16"];
4404            assert_eq!(
4405                generated_names, expected_names,
4406                "Data type names were not generated correctly for union_fixed_dur_decfix"
4407            );
4408            expected_cols.push(arr);
4409        }
4410        // 6) union_enum_records_array_map: [enum ColorU, record RecA, record RecB, array<long>, map<string>]
4411        {
4412            let (uf, _) = get_union("union_enum_records_array_map");
4413            let tid_enum = tid_by_dt(&uf, |dt| matches!(dt, DataType::Dictionary(_, _)));
4414            let tid_reca = tid_by_dt(&uf, |dt| {
4415                if let DataType::Struct(fs) = dt {
4416                    fs.len() == 2 && fs[0].name() == "a" && fs[1].name() == "b"
4417                } else {
4418                    false
4419                }
4420            });
4421            let tid_recb = tid_by_dt(&uf, |dt| {
4422                if let DataType::Struct(fs) = dt {
4423                    fs.len() == 2 && fs[0].name() == "x" && fs[1].name() == "y"
4424                } else {
4425                    false
4426                }
4427            });
4428            let tid_arr = tid_by_dt(&uf, |dt| matches!(dt, DataType::List(_)));
4429            let tids = vec![tid_enum, tid_reca, tid_recb, tid_arr];
4430            let offs = vec![0, 0, 0, 0];
4431            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4432                DataType::Dictionary(_, _) => {
4433                    let keys = Int32Array::from(vec![0i32]); // "RED"
4434                    let values =
4435                        Arc::new(StringArray::from(vec!["RED", "GREEN", "BLUE"])) as ArrayRef;
4436                    Some(
4437                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
4438                            as ArrayRef,
4439                    )
4440                }
4441                DataType::Struct(fs)
4442                    if fs.len() == 2 && fs[0].name() == "a" && fs[1].name() == "b" =>
4443                {
4444                    let a = Int32Array::from(vec![7]);
4445                    let b = StringArray::from(vec!["x"]);
4446                    Some(Arc::new(StructArray::new(
4447                        fs.clone(),
4448                        vec![Arc::new(a), Arc::new(b)],
4449                        None,
4450                    )) as ArrayRef)
4451                }
4452                DataType::Struct(fs)
4453                    if fs.len() == 2 && fs[0].name() == "x" && fs[1].name() == "y" =>
4454                {
4455                    let x = Int64Array::from(vec![123_456_789i64]);
4456                    let y = BinaryArray::from(vec![&[0xFF, 0x00][..]]);
4457                    Some(Arc::new(StructArray::new(
4458                        fs.clone(),
4459                        vec![Arc::new(x), Arc::new(y)],
4460                        None,
4461                    )) as ArrayRef)
4462                }
4463                DataType::List(field) => {
4464                    let values = Int64Array::from(vec![1i64, 2, 3]);
4465                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3]));
4466                    Some(Arc::new(
4467                        ListArray::try_new(field.clone(), offsets, Arc::new(values), None).unwrap(),
4468                    ) as ArrayRef)
4469                }
4470                DataType::Map(_, _) => None,
4471                other => panic!("unexpected child {other:?}"),
4472            });
4473            expected_cols.push(arr);
4474        }
4475        // 7) union_date_or_fixed4: [date32, fixed(4)]
4476        {
4477            let (uf, _) = get_union("union_date_or_fixed4");
4478            let tid_date = tid_by_dt(&uf, |dt| matches!(dt, DataType::Date32));
4479            let tid_fx4 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(4)));
4480            let tids = vec![tid_date, tid_fx4, tid_date, tid_fx4];
4481            let offs = vec![0, 0, 1, 1];
4482            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4483                DataType::Date32 => {
4484                    Some(Arc::new(arrow_array::Date32Array::from(vec![date_a, 0])) as ArrayRef)
4485                }
4486                DataType::FixedSizeBinary(4) => {
4487                    let it = [Some(fx4_abcd), Some(fx4_misc)].into_iter();
4488                    Some(Arc::new(
4489                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 4).unwrap(),
4490                    ) as ArrayRef)
4491                }
4492                _ => None,
4493            });
4494            expected_cols.push(arr);
4495        }
4496        // 8) union_time_millis_or_enum: [time-millis, enum OnOff]
4497        {
4498            let (uf, _) = get_union("union_time_millis_or_enum");
4499            let tid_ms = tid_by_dt(&uf, |dt| {
4500                matches!(dt, DataType::Time32(arrow_schema::TimeUnit::Millisecond))
4501            });
4502            let tid_en = tid_by_dt(&uf, |dt| matches!(dt, DataType::Dictionary(_, _)));
4503            let tids = vec![tid_ms, tid_en, tid_en, tid_ms];
4504            let offs = vec![0, 0, 1, 1];
4505            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4506                DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
4507                    Some(Arc::new(Time32MillisecondArray::from(vec![time_ms_a, 0])) as ArrayRef)
4508                }
4509                DataType::Dictionary(_, _) => {
4510                    let keys = Int32Array::from(vec![0i32, 1]); // "ON", "OFF"
4511                    let values = Arc::new(StringArray::from(vec!["ON", "OFF"])) as ArrayRef;
4512                    Some(
4513                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
4514                            as ArrayRef,
4515                    )
4516                }
4517                _ => None,
4518            });
4519            expected_cols.push(arr);
4520        }
4521        // 9) union_time_micros_or_string: [time-micros, string]
4522        {
4523            let (uf, _) = get_union("union_time_micros_or_string");
4524            let tid_us = tid_by_dt(&uf, |dt| {
4525                matches!(dt, DataType::Time64(arrow_schema::TimeUnit::Microsecond))
4526            });
4527            let tid_s = tid_by_name(&uf, "string");
4528            let tids = vec![tid_s, tid_us, tid_s, tid_s];
4529            let offs = vec![0, 0, 1, 2];
4530            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4531                DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
4532                    Some(Arc::new(Time64MicrosecondArray::from(vec![time_us_b])) as ArrayRef)
4533                }
4534                DataType::Utf8 => {
4535                    Some(Arc::new(StringArray::from(vec!["evening", "night", ""])) as ArrayRef)
4536                }
4537                _ => None,
4538            });
4539            expected_cols.push(arr);
4540        }
4541        // 10) union_ts_millis_utc_or_array: [timestamp-millis(TZ), array<int>]
4542        {
4543            let (uf, _) = get_union("union_ts_millis_utc_or_array");
4544            let tid_ts = tid_by_dt(&uf, |dt| {
4545                matches!(
4546                    dt,
4547                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, _)
4548                )
4549            });
4550            let tid_arr = tid_by_dt(&uf, |dt| matches!(dt, DataType::List(_)));
4551            let tids = vec![tid_ts, tid_arr, tid_arr, tid_ts];
4552            let offs = vec![0, 0, 1, 1];
4553            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4554                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
4555                    let a = TimestampMillisecondArray::from(vec![
4556                        ts_ms_2024_01_01,
4557                        ts_ms_2024_01_01 + 86_400_000,
4558                    ]);
4559                    Some(Arc::new(if let Some(tz) = tz {
4560                        a.with_timezone(tz.clone())
4561                    } else {
4562                        a
4563                    }) as ArrayRef)
4564                }
4565                DataType::List(field) => {
4566                    let values = Int32Array::from(vec![0, 1, 2, -1, 0, 1]);
4567                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 6]));
4568                    Some(Arc::new(
4569                        ListArray::try_new(field.clone(), offsets, Arc::new(values), None).unwrap(),
4570                    ) as ArrayRef)
4571                }
4572                _ => None,
4573            });
4574            expected_cols.push(arr);
4575        }
4576        // 11) union_ts_micros_local_or_bytes: [local-timestamp-micros, bytes]
4577        {
4578            let (uf, _) = get_union("union_ts_micros_local_or_bytes");
4579            let tid_lts = tid_by_dt(&uf, |dt| {
4580                matches!(
4581                    dt,
4582                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None)
4583                )
4584            });
4585            let tid_b = tid_by_name(&uf, "bytes");
4586            let tids = vec![tid_b, tid_lts, tid_b, tid_b];
4587            let offs = vec![0, 0, 1, 2];
4588            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4589                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None) => Some(Arc::new(
4590                    TimestampMicrosecondArray::from(vec![ts_us_2024_01_01]),
4591                )
4592                    as ArrayRef),
4593                DataType::Binary => Some(Arc::new(BinaryArray::from(vec![
4594                    &b"\x11\x22\x33"[..],
4595                    &b"\x00"[..],
4596                    &b"\x10\x20\x30\x40"[..],
4597                ])) as ArrayRef),
4598                _ => None,
4599            });
4600            expected_cols.push(arr);
4601        }
4602        // 12) union_uuid_or_fixed10: [uuid(string)->fixed(16), fixed(10)]
4603        {
4604            let (uf, _) = get_union("union_uuid_or_fixed10");
4605            let tid_fx16 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(16)));
4606            let tid_fx10 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(10)));
4607            let tids = vec![tid_fx16, tid_fx10, tid_fx16, tid_fx10];
4608            let offs = vec![0, 0, 1, 1];
4609            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4610                DataType::FixedSizeBinary(16) => {
4611                    let it = [Some(uuid1), Some(uuid2)].into_iter();
4612                    Some(Arc::new(
4613                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
4614                    ) as ArrayRef)
4615                }
4616                DataType::FixedSizeBinary(10) => {
4617                    let it = [Some(fx10_ascii), Some(fx10_aa)].into_iter();
4618                    Some(Arc::new(
4619                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 10).unwrap(),
4620                    ) as ArrayRef)
4621                }
4622                _ => None,
4623            });
4624            expected_cols.push(arr);
4625        }
4626        // 13) union_dec_bytes_or_dec_fixed: [bytes dec(10,2), fixed(20) dec(20,4)]
4627        {
4628            let (uf, _) = get_union("union_dec_bytes_or_dec_fixed");
4629            let tid_b10s2 = tid_by_dt(&uf, |dt| match dt {
4630                #[cfg(feature = "small_decimals")]
4631                DataType::Decimal64(10, 2) => true,
4632                DataType::Decimal128(10, 2) | DataType::Decimal256(10, 2) => true,
4633                _ => false,
4634            });
4635            let tid_f20s4 = tid_by_dt(&uf, |dt| {
4636                matches!(
4637                    dt,
4638                    DataType::Decimal128(20, 4) | DataType::Decimal256(20, 4)
4639                )
4640            });
4641            let tids = vec![tid_b10s2, tid_f20s4, tid_b10s2, tid_f20s4];
4642            let offs = vec![0, 0, 1, 1];
4643            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4644                #[cfg(feature = "small_decimals")]
4645                DataType::Decimal64(10, 2) => {
4646                    let a = Decimal64Array::from_iter_values([dec_b_scale2_pos as i64, 0i64]);
4647                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4648                }
4649                DataType::Decimal128(10, 2) => {
4650                    let a = Decimal128Array::from_iter_values([dec_b_scale2_pos, 0]);
4651                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4652                }
4653                DataType::Decimal256(10, 2) => {
4654                    let a = Decimal256Array::from_iter_values([
4655                        i256::from_i128(dec_b_scale2_pos),
4656                        i256::from(0),
4657                    ]);
4658                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4659                }
4660                DataType::Decimal128(20, 4) => {
4661                    let a = Decimal128Array::from_iter_values([dec_fix20_s4_neg, dec_fix20_s4]);
4662                    Some(Arc::new(a.with_precision_and_scale(20, 4).unwrap()) as ArrayRef)
4663                }
4664                DataType::Decimal256(20, 4) => {
4665                    let a = Decimal256Array::from_iter_values([
4666                        i256::from_i128(dec_fix20_s4_neg),
4667                        i256::from_i128(dec_fix20_s4),
4668                    ]);
4669                    Some(Arc::new(a.with_precision_and_scale(20, 4).unwrap()) as ArrayRef)
4670                }
4671                _ => None,
4672            });
4673            expected_cols.push(arr);
4674        }
4675        // 14) union_null_bytes_string: ["null","bytes","string"]
4676        {
4677            let (uf, _) = get_union("union_null_bytes_string");
4678            let tid_n = tid_by_name(&uf, "null");
4679            let tid_b = tid_by_name(&uf, "bytes");
4680            let tid_s = tid_by_name(&uf, "string");
4681            let tids = vec![tid_n, tid_b, tid_s, tid_s];
4682            let offs = vec![0, 0, 0, 1];
4683            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4684                "null" => Some(Arc::new(arrow_array::NullArray::new(1)) as ArrayRef),
4685                "bytes" => Some(Arc::new(BinaryArray::from(vec![&b"\x01\x02"[..]])) as ArrayRef),
4686                "string" => Some(Arc::new(StringArray::from(vec!["text", "u"])) as ArrayRef),
4687                _ => None,
4688            });
4689            expected_cols.push(arr);
4690        }
4691        // 15) array_of_union: array<[long,string]>
4692        {
4693            let idx = schema.index_of("array_of_union").unwrap();
4694            let dt = schema.field(idx).data_type().clone();
4695            let (item_field, _) = match &dt {
4696                DataType::List(f) => (f.clone(), ()),
4697                other => panic!("array_of_union must be List, got {other:?}"),
4698            };
4699            let (uf, _) = match item_field.data_type() {
4700                DataType::Union(f, m) => (f.clone(), m),
4701                other => panic!("array_of_union items must be Union, got {other:?}"),
4702            };
4703            let tid_l = tid_by_name(&uf, "long");
4704            let tid_s = tid_by_name(&uf, "string");
4705            let type_ids = vec![tid_l, tid_s, tid_l, tid_s, tid_l, tid_l, tid_s, tid_l];
4706            let offsets = vec![0, 0, 1, 1, 2, 3, 2, 4];
4707            let values_union =
4708                mk_dense_union(&uf, type_ids, offsets, |f| match f.name().as_str() {
4709                    "long" => {
4710                        Some(Arc::new(Int64Array::from(vec![1i64, -5, 42, -1, 0])) as ArrayRef)
4711                    }
4712                    "string" => Some(Arc::new(StringArray::from(vec!["a", "", "z"])) as ArrayRef),
4713                    _ => None,
4714                });
4715            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 5, 6, 8]));
4716            expected_cols.push(Arc::new(
4717                ListArray::try_new(item_field.clone(), list_offsets, values_union, None).unwrap(),
4718            ));
4719        }
4720        // 16) map_of_union: map<[null,double]>
4721        {
4722            let idx = schema.index_of("map_of_union").unwrap();
4723            let dt = schema.field(idx).data_type().clone();
4724            let (entry_field, ordered) = match &dt {
4725                DataType::Map(f, ordered) => (f.clone(), *ordered),
4726                other => panic!("map_of_union must be Map, got {other:?}"),
4727            };
4728            let DataType::Struct(entry_fields) = entry_field.data_type() else {
4729                panic!("map entries must be struct")
4730            };
4731            let key_field = entry_fields[0].clone();
4732            let val_field = entry_fields[1].clone();
4733            let keys = StringArray::from(vec!["a", "b", "x", "pi"]);
4734            let rounded_pi = (std::f64::consts::PI * 100_000.0).round() / 100_000.0;
4735            let values: ArrayRef = match val_field.data_type() {
4736                DataType::Union(uf, _) => {
4737                    let tid_n = tid_by_name(uf, "null");
4738                    let tid_d = tid_by_name(uf, "double");
4739                    let tids = vec![tid_n, tid_d, tid_d, tid_d];
4740                    let offs = vec![0, 0, 1, 2];
4741                    mk_dense_union(uf, tids, offs, |f| match f.name().as_str() {
4742                        "null" => Some(Arc::new(NullArray::new(1)) as ArrayRef),
4743                        "double" => Some(Arc::new(arrow_array::Float64Array::from(vec![
4744                            2.5f64, -0.5f64, rounded_pi,
4745                        ])) as ArrayRef),
4746                        _ => None,
4747                    })
4748                }
4749                DataType::Float64 => Arc::new(arrow_array::Float64Array::from(vec![
4750                    None,
4751                    Some(2.5),
4752                    Some(-0.5),
4753                    Some(rounded_pi),
4754                ])),
4755                other => panic!("unexpected map value type {other:?}"),
4756            };
4757            let entries = StructArray::new(
4758                Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
4759                vec![Arc::new(keys) as ArrayRef, values],
4760                None,
4761            );
4762            let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3, 3, 4]));
4763            expected_cols.push(Arc::new(MapArray::new(
4764                entry_field,
4765                offsets,
4766                entries,
4767                None,
4768                ordered,
4769            )));
4770        }
4771        // 17) record_with_union_field: struct { id:int, u:[int,string] }
4772        {
4773            let idx = schema.index_of("record_with_union_field").unwrap();
4774            let DataType::Struct(rec_fields) = schema.field(idx).data_type() else {
4775                panic!("record_with_union_field should be Struct")
4776            };
4777            let id = Int32Array::from(vec![1, 2, 3, 4]);
4778            let u_field = rec_fields.iter().find(|f| f.name() == "u").unwrap();
4779            let DataType::Union(uf, _) = u_field.data_type() else {
4780                panic!("u must be Union")
4781            };
4782            let tid_i = tid_by_name(uf, "int");
4783            let tid_s = tid_by_name(uf, "string");
4784            let tids = vec![tid_s, tid_i, tid_i, tid_s];
4785            let offs = vec![0, 0, 1, 1];
4786            let u = mk_dense_union(uf, tids, offs, |f| match f.name().as_str() {
4787                "int" => Some(Arc::new(Int32Array::from(vec![99, 0])) as ArrayRef),
4788                "string" => Some(Arc::new(StringArray::from(vec!["one", "four"])) as ArrayRef),
4789                _ => None,
4790            });
4791            let rec = StructArray::new(rec_fields.clone(), vec![Arc::new(id) as ArrayRef, u], None);
4792            expected_cols.push(Arc::new(rec));
4793        }
4794        // 18) union_ts_micros_utc_or_map: [timestamp-micros(TZ), map<long>]
4795        {
4796            let (uf, _) = get_union("union_ts_micros_utc_or_map");
4797            let tid_ts = tid_by_dt(&uf, |dt| {
4798                matches!(
4799                    dt,
4800                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, Some(_))
4801                )
4802            });
4803            let tid_map = tid_by_dt(&uf, |dt| matches!(dt, DataType::Map(_, _)));
4804            let tids = vec![tid_ts, tid_map, tid_ts, tid_map];
4805            let offs = vec![0, 0, 1, 1];
4806            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4807                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
4808                    let a = TimestampMicrosecondArray::from(vec![ts_us_2024_01_01, 0i64]);
4809                    Some(Arc::new(if let Some(tz) = tz {
4810                        a.with_timezone(tz.clone())
4811                    } else {
4812                        a
4813                    }) as ArrayRef)
4814                }
4815                DataType::Map(entry_field, ordered) => {
4816                    let DataType::Struct(fs) = entry_field.data_type() else {
4817                        panic!("map entries must be struct")
4818                    };
4819                    let key_field = fs[0].clone();
4820                    let val_field = fs[1].clone();
4821                    assert_eq!(key_field.data_type(), &DataType::Utf8);
4822                    assert_eq!(val_field.data_type(), &DataType::Int64);
4823                    let keys = StringArray::from(vec!["k1", "k2", "n"]);
4824                    let vals = Int64Array::from(vec![1i64, 2, 0]);
4825                    let entries = StructArray::new(
4826                        Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
4827                        vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
4828                        None,
4829                    );
4830                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3]));
4831                    Some(Arc::new(MapArray::new(
4832                        entry_field.clone(),
4833                        offsets,
4834                        entries,
4835                        None,
4836                        *ordered,
4837                    )) as ArrayRef)
4838                }
4839                _ => None,
4840            });
4841            expected_cols.push(arr);
4842        }
4843        // 19) union_ts_millis_local_or_string: [local-timestamp-millis, string]
4844        {
4845            let (uf, _) = get_union("union_ts_millis_local_or_string");
4846            let tid_ts = tid_by_dt(&uf, |dt| {
4847                matches!(
4848                    dt,
4849                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None)
4850                )
4851            });
4852            let tid_s = tid_by_name(&uf, "string");
4853            let tids = vec![tid_s, tid_ts, tid_s, tid_s];
4854            let offs = vec![0, 0, 1, 2];
4855            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4856                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None) => Some(Arc::new(
4857                    TimestampMillisecondArray::from(vec![ts_ms_2024_01_01]),
4858                )
4859                    as ArrayRef),
4860                DataType::Utf8 => {
4861                    Some(
4862                        Arc::new(StringArray::from(vec!["local midnight", "done", ""])) as ArrayRef,
4863                    )
4864                }
4865                _ => None,
4866            });
4867            expected_cols.push(arr);
4868        }
4869        // 20) union_bool_or_string: ["boolean","string"]
4870        {
4871            let (uf, _) = get_union("union_bool_or_string");
4872            let tid_b = tid_by_name(&uf, "boolean");
4873            let tid_s = tid_by_name(&uf, "string");
4874            let tids = vec![tid_b, tid_s, tid_b, tid_s];
4875            let offs = vec![0, 0, 1, 1];
4876            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4877                "boolean" => Some(Arc::new(BooleanArray::from(vec![true, false])) as ArrayRef),
4878                "string" => Some(Arc::new(StringArray::from(vec!["no", "yes"])) as ArrayRef),
4879                _ => None,
4880            });
4881            expected_cols.push(arr);
4882        }
4883        let expected = RecordBatch::try_new(schema.clone(), expected_cols).unwrap();
4884        assert_eq!(
4885            actual, expected,
4886            "full end-to-end equality for union_fields.avro"
4887        );
4888    }
4889
4890    #[test]
4891    fn test_read_zero_byte_avro_file() {
4892        let batch = read_file("test/data/zero_byte.avro", 3, false);
4893        let schema = batch.schema();
4894        assert_eq!(schema.fields().len(), 1);
4895        let field = schema.field(0);
4896        assert_eq!(field.name(), "data");
4897        assert_eq!(field.data_type(), &DataType::Binary);
4898        assert!(field.is_nullable());
4899        assert_eq!(batch.num_rows(), 3);
4900        assert_eq!(batch.num_columns(), 1);
4901        let binary_array = batch
4902            .column(0)
4903            .as_any()
4904            .downcast_ref::<BinaryArray>()
4905            .unwrap();
4906        assert!(binary_array.is_null(0));
4907        assert!(binary_array.is_valid(1));
4908        assert_eq!(binary_array.value(1), b"");
4909        assert!(binary_array.is_valid(2));
4910        assert_eq!(binary_array.value(2), b"some bytes");
4911    }
4912
4913    #[test]
4914    fn test_alltypes() {
4915        let expected = RecordBatch::try_from_iter_with_nullable([
4916            (
4917                "id",
4918                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
4919                true,
4920            ),
4921            (
4922                "bool_col",
4923                Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
4924                true,
4925            ),
4926            (
4927                "tinyint_col",
4928                Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
4929                true,
4930            ),
4931            (
4932                "smallint_col",
4933                Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
4934                true,
4935            ),
4936            (
4937                "int_col",
4938                Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
4939                true,
4940            ),
4941            (
4942                "bigint_col",
4943                Arc::new(Int64Array::from_iter_values((0..8).map(|x| (x % 2) * 10))) as _,
4944                true,
4945            ),
4946            (
4947                "float_col",
4948                Arc::new(Float32Array::from_iter_values(
4949                    (0..8).map(|x| (x % 2) as f32 * 1.1),
4950                )) as _,
4951                true,
4952            ),
4953            (
4954                "double_col",
4955                Arc::new(Float64Array::from_iter_values(
4956                    (0..8).map(|x| (x % 2) as f64 * 10.1),
4957                )) as _,
4958                true,
4959            ),
4960            (
4961                "date_string_col",
4962                Arc::new(BinaryArray::from_iter_values([
4963                    [48, 51, 47, 48, 49, 47, 48, 57],
4964                    [48, 51, 47, 48, 49, 47, 48, 57],
4965                    [48, 52, 47, 48, 49, 47, 48, 57],
4966                    [48, 52, 47, 48, 49, 47, 48, 57],
4967                    [48, 50, 47, 48, 49, 47, 48, 57],
4968                    [48, 50, 47, 48, 49, 47, 48, 57],
4969                    [48, 49, 47, 48, 49, 47, 48, 57],
4970                    [48, 49, 47, 48, 49, 47, 48, 57],
4971                ])) as _,
4972                true,
4973            ),
4974            (
4975                "string_col",
4976                Arc::new(BinaryArray::from_iter_values((0..8).map(|x| [48 + x % 2]))) as _,
4977                true,
4978            ),
4979            (
4980                "timestamp_col",
4981                Arc::new(
4982                    TimestampMicrosecondArray::from_iter_values([
4983                        1235865600000000, // 2009-03-01T00:00:00.000
4984                        1235865660000000, // 2009-03-01T00:01:00.000
4985                        1238544000000000, // 2009-04-01T00:00:00.000
4986                        1238544060000000, // 2009-04-01T00:01:00.000
4987                        1233446400000000, // 2009-02-01T00:00:00.000
4988                        1233446460000000, // 2009-02-01T00:01:00.000
4989                        1230768000000000, // 2009-01-01T00:00:00.000
4990                        1230768060000000, // 2009-01-01T00:01:00.000
4991                    ])
4992                    .with_timezone("+00:00"),
4993                ) as _,
4994                true,
4995            ),
4996        ])
4997        .unwrap();
4998
4999        for file in files() {
5000            let file = arrow_test_data(file);
5001
5002            assert_eq!(read_file(&file, 8, false), expected);
5003            assert_eq!(read_file(&file, 3, false), expected);
5004        }
5005    }
5006
5007    #[test]
5008    // TODO: avoid requiring snappy for this file
5009    #[cfg(feature = "snappy")]
5010    fn test_alltypes_dictionary() {
5011        let file = "avro/alltypes_dictionary.avro";
5012        let expected = RecordBatch::try_from_iter_with_nullable([
5013            ("id", Arc::new(Int32Array::from(vec![0, 1])) as _, true),
5014            (
5015                "bool_col",
5016                Arc::new(BooleanArray::from(vec![Some(true), Some(false)])) as _,
5017                true,
5018            ),
5019            (
5020                "tinyint_col",
5021                Arc::new(Int32Array::from(vec![0, 1])) as _,
5022                true,
5023            ),
5024            (
5025                "smallint_col",
5026                Arc::new(Int32Array::from(vec![0, 1])) as _,
5027                true,
5028            ),
5029            ("int_col", Arc::new(Int32Array::from(vec![0, 1])) as _, true),
5030            (
5031                "bigint_col",
5032                Arc::new(Int64Array::from(vec![0, 10])) as _,
5033                true,
5034            ),
5035            (
5036                "float_col",
5037                Arc::new(Float32Array::from(vec![0.0, 1.1])) as _,
5038                true,
5039            ),
5040            (
5041                "double_col",
5042                Arc::new(Float64Array::from(vec![0.0, 10.1])) as _,
5043                true,
5044            ),
5045            (
5046                "date_string_col",
5047                Arc::new(BinaryArray::from_iter_values([b"01/01/09", b"01/01/09"])) as _,
5048                true,
5049            ),
5050            (
5051                "string_col",
5052                Arc::new(BinaryArray::from_iter_values([b"0", b"1"])) as _,
5053                true,
5054            ),
5055            (
5056                "timestamp_col",
5057                Arc::new(
5058                    TimestampMicrosecondArray::from_iter_values([
5059                        1230768000000000, // 2009-01-01T00:00:00.000
5060                        1230768060000000, // 2009-01-01T00:01:00.000
5061                    ])
5062                    .with_timezone("+00:00"),
5063                ) as _,
5064                true,
5065            ),
5066        ])
5067        .unwrap();
5068        let file_path = arrow_test_data(file);
5069        let batch_large = read_file(&file_path, 8, false);
5070        assert_eq!(
5071            batch_large, expected,
5072            "Decoded RecordBatch does not match for file {file}"
5073        );
5074        let batch_small = read_file(&file_path, 3, false);
5075        assert_eq!(
5076            batch_small, expected,
5077            "Decoded RecordBatch (batch size 3) does not match for file {file}"
5078        );
5079    }
5080
5081    #[test]
5082    fn test_alltypes_nulls_plain() {
5083        let file = "avro/alltypes_nulls_plain.avro";
5084        let expected = RecordBatch::try_from_iter_with_nullable([
5085            (
5086                "string_col",
5087                Arc::new(StringArray::from(vec![None::<&str>])) as _,
5088                true,
5089            ),
5090            ("int_col", Arc::new(Int32Array::from(vec![None])) as _, true),
5091            (
5092                "bool_col",
5093                Arc::new(BooleanArray::from(vec![None])) as _,
5094                true,
5095            ),
5096            (
5097                "bigint_col",
5098                Arc::new(Int64Array::from(vec![None])) as _,
5099                true,
5100            ),
5101            (
5102                "float_col",
5103                Arc::new(Float32Array::from(vec![None])) as _,
5104                true,
5105            ),
5106            (
5107                "double_col",
5108                Arc::new(Float64Array::from(vec![None])) as _,
5109                true,
5110            ),
5111            (
5112                "bytes_col",
5113                Arc::new(BinaryArray::from(vec![None::<&[u8]>])) as _,
5114                true,
5115            ),
5116        ])
5117        .unwrap();
5118        let file_path = arrow_test_data(file);
5119        let batch_large = read_file(&file_path, 8, false);
5120        assert_eq!(
5121            batch_large, expected,
5122            "Decoded RecordBatch does not match for file {file}"
5123        );
5124        let batch_small = read_file(&file_path, 3, false);
5125        assert_eq!(
5126            batch_small, expected,
5127            "Decoded RecordBatch (batch size 3) does not match for file {file}"
5128        );
5129    }
5130
5131    #[test]
5132    // TODO: avoid requiring snappy for this file
5133    #[cfg(feature = "snappy")]
5134    fn test_binary() {
5135        let file = arrow_test_data("avro/binary.avro");
5136        let batch = read_file(&file, 8, false);
5137        let expected = RecordBatch::try_from_iter_with_nullable([(
5138            "foo",
5139            Arc::new(BinaryArray::from_iter_values(vec![
5140                b"\x00" as &[u8],
5141                b"\x01" as &[u8],
5142                b"\x02" as &[u8],
5143                b"\x03" as &[u8],
5144                b"\x04" as &[u8],
5145                b"\x05" as &[u8],
5146                b"\x06" as &[u8],
5147                b"\x07" as &[u8],
5148                b"\x08" as &[u8],
5149                b"\t" as &[u8],
5150                b"\n" as &[u8],
5151                b"\x0b" as &[u8],
5152            ])) as Arc<dyn Array>,
5153            true,
5154        )])
5155        .unwrap();
5156        assert_eq!(batch, expected);
5157    }
5158
5159    #[test]
5160    // TODO: avoid requiring snappy for these files
5161    #[cfg(feature = "snappy")]
5162    fn test_decimal() {
5163        // Choose expected Arrow types depending on the `small_decimals` feature flag.
5164        // With `small_decimals` enabled, Decimal32/Decimal64 are used where their
5165        // precision allows; otherwise, those cases resolve to Decimal128.
5166        #[cfg(feature = "small_decimals")]
5167        let files: [(&str, DataType, HashMap<String, String>); 8] = [
5168            (
5169                "avro/fixed_length_decimal.avro",
5170                DataType::Decimal128(25, 2),
5171                HashMap::from([
5172                    (
5173                        "avro.namespace".to_string(),
5174                        "topLevelRecord.value".to_string(),
5175                    ),
5176                    ("avro.name".to_string(), "fixed".to_string()),
5177                ]),
5178            ),
5179            (
5180                "avro/fixed_length_decimal_legacy.avro",
5181                DataType::Decimal64(13, 2),
5182                HashMap::from([
5183                    (
5184                        "avro.namespace".to_string(),
5185                        "topLevelRecord.value".to_string(),
5186                    ),
5187                    ("avro.name".to_string(), "fixed".to_string()),
5188                ]),
5189            ),
5190            (
5191                "avro/int32_decimal.avro",
5192                DataType::Decimal32(4, 2),
5193                HashMap::from([
5194                    (
5195                        "avro.namespace".to_string(),
5196                        "topLevelRecord.value".to_string(),
5197                    ),
5198                    ("avro.name".to_string(), "fixed".to_string()),
5199                ]),
5200            ),
5201            (
5202                "avro/int64_decimal.avro",
5203                DataType::Decimal64(10, 2),
5204                HashMap::from([
5205                    (
5206                        "avro.namespace".to_string(),
5207                        "topLevelRecord.value".to_string(),
5208                    ),
5209                    ("avro.name".to_string(), "fixed".to_string()),
5210                ]),
5211            ),
5212            (
5213                "test/data/int256_decimal.avro",
5214                DataType::Decimal256(76, 10),
5215                HashMap::new(),
5216            ),
5217            (
5218                "test/data/fixed256_decimal.avro",
5219                DataType::Decimal256(76, 10),
5220                HashMap::from([("avro.name".to_string(), "Decimal256Fixed".to_string())]),
5221            ),
5222            (
5223                "test/data/fixed_length_decimal_legacy_32.avro",
5224                DataType::Decimal32(9, 2),
5225                HashMap::from([("avro.name".to_string(), "Decimal32FixedLegacy".to_string())]),
5226            ),
5227            (
5228                "test/data/int128_decimal.avro",
5229                DataType::Decimal128(38, 2),
5230                HashMap::new(),
5231            ),
5232        ];
5233        #[cfg(not(feature = "small_decimals"))]
5234        let files: [(&str, DataType, HashMap<String, String>); 8] = [
5235            (
5236                "avro/fixed_length_decimal.avro",
5237                DataType::Decimal128(25, 2),
5238                HashMap::from([
5239                    (
5240                        "avro.namespace".to_string(),
5241                        "topLevelRecord.value".to_string(),
5242                    ),
5243                    ("avro.name".to_string(), "fixed".to_string()),
5244                ]),
5245            ),
5246            (
5247                "avro/fixed_length_decimal_legacy.avro",
5248                DataType::Decimal128(13, 2),
5249                HashMap::from([
5250                    (
5251                        "avro.namespace".to_string(),
5252                        "topLevelRecord.value".to_string(),
5253                    ),
5254                    ("avro.name".to_string(), "fixed".to_string()),
5255                ]),
5256            ),
5257            (
5258                "avro/int32_decimal.avro",
5259                DataType::Decimal128(4, 2),
5260                HashMap::from([
5261                    (
5262                        "avro.namespace".to_string(),
5263                        "topLevelRecord.value".to_string(),
5264                    ),
5265                    ("avro.name".to_string(), "fixed".to_string()),
5266                ]),
5267            ),
5268            (
5269                "avro/int64_decimal.avro",
5270                DataType::Decimal128(10, 2),
5271                HashMap::from([
5272                    (
5273                        "avro.namespace".to_string(),
5274                        "topLevelRecord.value".to_string(),
5275                    ),
5276                    ("avro.name".to_string(), "fixed".to_string()),
5277                ]),
5278            ),
5279            (
5280                "test/data/int256_decimal.avro",
5281                DataType::Decimal256(76, 10),
5282                HashMap::new(),
5283            ),
5284            (
5285                "test/data/fixed256_decimal.avro",
5286                DataType::Decimal256(76, 10),
5287                HashMap::from([("avro.name".to_string(), "Decimal256Fixed".to_string())]),
5288            ),
5289            (
5290                "test/data/fixed_length_decimal_legacy_32.avro",
5291                DataType::Decimal128(9, 2),
5292                HashMap::from([("avro.name".to_string(), "Decimal32FixedLegacy".to_string())]),
5293            ),
5294            (
5295                "test/data/int128_decimal.avro",
5296                DataType::Decimal128(38, 2),
5297                HashMap::new(),
5298            ),
5299        ];
5300        for (file, expected_dt, mut metadata) in files {
5301            let (precision, scale) = match expected_dt {
5302                DataType::Decimal32(p, s)
5303                | DataType::Decimal64(p, s)
5304                | DataType::Decimal128(p, s)
5305                | DataType::Decimal256(p, s) => (p, s),
5306                _ => unreachable!("Unexpected decimal type in test inputs"),
5307            };
5308            assert!(scale >= 0, "test data uses non-negative scales only");
5309            let scale_u32 = scale as u32;
5310            let file_path: String = if file.starts_with("avro/") {
5311                arrow_test_data(file)
5312            } else {
5313                std::path::PathBuf::from(env!("CARGO_MANIFEST_DIR"))
5314                    .join(file)
5315                    .to_string_lossy()
5316                    .into_owned()
5317            };
5318            let pow10: i128 = 10i128.pow(scale_u32);
5319            let values_i128: Vec<i128> = (1..=24).map(|n| (n as i128) * pow10).collect();
5320            let build_expected = |dt: &DataType, values: &[i128]| -> ArrayRef {
5321                match *dt {
5322                    #[cfg(feature = "small_decimals")]
5323                    DataType::Decimal32(p, s) => {
5324                        let it = values.iter().map(|&v| v as i32);
5325                        Arc::new(
5326                            Decimal32Array::from_iter_values(it)
5327                                .with_precision_and_scale(p, s)
5328                                .unwrap(),
5329                        )
5330                    }
5331                    #[cfg(feature = "small_decimals")]
5332                    DataType::Decimal64(p, s) => {
5333                        let it = values.iter().map(|&v| v as i64);
5334                        Arc::new(
5335                            Decimal64Array::from_iter_values(it)
5336                                .with_precision_and_scale(p, s)
5337                                .unwrap(),
5338                        )
5339                    }
5340                    DataType::Decimal128(p, s) => {
5341                        let it = values.iter().copied();
5342                        Arc::new(
5343                            Decimal128Array::from_iter_values(it)
5344                                .with_precision_and_scale(p, s)
5345                                .unwrap(),
5346                        )
5347                    }
5348                    DataType::Decimal256(p, s) => {
5349                        let it = values.iter().map(|&v| i256::from_i128(v));
5350                        Arc::new(
5351                            Decimal256Array::from_iter_values(it)
5352                                .with_precision_and_scale(p, s)
5353                                .unwrap(),
5354                        )
5355                    }
5356                    _ => unreachable!("Unexpected decimal type in test"),
5357                }
5358            };
5359            let actual_batch = read_file(&file_path, 8, false);
5360            let actual_nullable = actual_batch.schema().field(0).is_nullable();
5361            let expected_array = build_expected(&expected_dt, &values_i128);
5362            metadata.insert("precision".to_string(), precision.to_string());
5363            metadata.insert("scale".to_string(), scale.to_string());
5364            let field =
5365                Field::new("value", expected_dt.clone(), actual_nullable).with_metadata(metadata);
5366            let expected_schema = Arc::new(Schema::new(vec![field]));
5367            let expected_batch =
5368                RecordBatch::try_new(expected_schema.clone(), vec![expected_array]).unwrap();
5369            assert_eq!(
5370                actual_batch, expected_batch,
5371                "Decoded RecordBatch does not match for {file}"
5372            );
5373            let actual_batch_small = read_file(&file_path, 3, false);
5374            assert_eq!(
5375                actual_batch_small, expected_batch,
5376                "Decoded RecordBatch does not match for {file} with batch size 3"
5377            );
5378        }
5379    }
5380
5381    #[test]
5382    fn test_read_duration_logical_types_feature_toggle() -> Result<(), ArrowError> {
5383        let file_path = std::path::PathBuf::from(env!("CARGO_MANIFEST_DIR"))
5384            .join("test/data/duration_logical_types.avro")
5385            .to_string_lossy()
5386            .into_owned();
5387
5388        let actual_batch = read_file(&file_path, 4, false);
5389
5390        let expected_batch = {
5391            #[cfg(feature = "avro_custom_types")]
5392            {
5393                let schema = Arc::new(Schema::new(vec![
5394                    Field::new(
5395                        "duration_time_nanos",
5396                        DataType::Duration(TimeUnit::Nanosecond),
5397                        false,
5398                    ),
5399                    Field::new(
5400                        "duration_time_micros",
5401                        DataType::Duration(TimeUnit::Microsecond),
5402                        false,
5403                    ),
5404                    Field::new(
5405                        "duration_time_millis",
5406                        DataType::Duration(TimeUnit::Millisecond),
5407                        false,
5408                    ),
5409                    Field::new(
5410                        "duration_time_seconds",
5411                        DataType::Duration(TimeUnit::Second),
5412                        false,
5413                    ),
5414                ]));
5415
5416                let nanos = Arc::new(PrimitiveArray::<DurationNanosecondType>::from(vec![
5417                    10, 20, 30, 40,
5418                ])) as ArrayRef;
5419                let micros = Arc::new(PrimitiveArray::<DurationMicrosecondType>::from(vec![
5420                    100, 200, 300, 400,
5421                ])) as ArrayRef;
5422                let millis = Arc::new(PrimitiveArray::<DurationMillisecondType>::from(vec![
5423                    1000, 2000, 3000, 4000,
5424                ])) as ArrayRef;
5425                let seconds = Arc::new(PrimitiveArray::<DurationSecondType>::from(vec![1, 2, 3, 4]))
5426                    as ArrayRef;
5427
5428                RecordBatch::try_new(schema, vec![nanos, micros, millis, seconds])?
5429            }
5430            #[cfg(not(feature = "avro_custom_types"))]
5431            {
5432                let schema = Arc::new(Schema::new(vec![
5433                    Field::new("duration_time_nanos", DataType::Int64, false).with_metadata(
5434                        [(
5435                            "logicalType".to_string(),
5436                            "arrow.duration-nanos".to_string(),
5437                        )]
5438                        .into(),
5439                    ),
5440                    Field::new("duration_time_micros", DataType::Int64, false).with_metadata(
5441                        [(
5442                            "logicalType".to_string(),
5443                            "arrow.duration-micros".to_string(),
5444                        )]
5445                        .into(),
5446                    ),
5447                    Field::new("duration_time_millis", DataType::Int64, false).with_metadata(
5448                        [(
5449                            "logicalType".to_string(),
5450                            "arrow.duration-millis".to_string(),
5451                        )]
5452                        .into(),
5453                    ),
5454                    Field::new("duration_time_seconds", DataType::Int64, false).with_metadata(
5455                        [(
5456                            "logicalType".to_string(),
5457                            "arrow.duration-seconds".to_string(),
5458                        )]
5459                        .into(),
5460                    ),
5461                ]));
5462
5463                let nanos =
5464                    Arc::new(PrimitiveArray::<Int64Type>::from(vec![10, 20, 30, 40])) as ArrayRef;
5465                let micros = Arc::new(PrimitiveArray::<Int64Type>::from(vec![100, 200, 300, 400]))
5466                    as ArrayRef;
5467                let millis = Arc::new(PrimitiveArray::<Int64Type>::from(vec![
5468                    1000, 2000, 3000, 4000,
5469                ])) as ArrayRef;
5470                let seconds =
5471                    Arc::new(PrimitiveArray::<Int64Type>::from(vec![1, 2, 3, 4])) as ArrayRef;
5472
5473                RecordBatch::try_new(schema, vec![nanos, micros, millis, seconds])?
5474            }
5475        };
5476
5477        assert_eq!(actual_batch, expected_batch);
5478
5479        Ok(())
5480    }
5481
5482    #[test]
5483    // TODO: avoid requiring snappy for this file
5484    #[cfg(feature = "snappy")]
5485    fn test_dict_pages_offset_zero() {
5486        let file = arrow_test_data("avro/dict-page-offset-zero.avro");
5487        let batch = read_file(&file, 32, false);
5488        let num_rows = batch.num_rows();
5489        let expected_field = Int32Array::from(vec![Some(1552); num_rows]);
5490        let expected = RecordBatch::try_from_iter_with_nullable([(
5491            "l_partkey",
5492            Arc::new(expected_field) as Arc<dyn Array>,
5493            true,
5494        )])
5495        .unwrap();
5496        assert_eq!(batch, expected);
5497    }
5498
5499    #[test]
5500    // TODO: avoid requiring snappy for this file
5501    #[cfg(feature = "snappy")]
5502    fn test_list_columns() {
5503        let file = arrow_test_data("avro/list_columns.avro");
5504        let mut int64_list_builder = ListBuilder::new(Int64Builder::new());
5505        {
5506            {
5507                let values = int64_list_builder.values();
5508                values.append_value(1);
5509                values.append_value(2);
5510                values.append_value(3);
5511            }
5512            int64_list_builder.append(true);
5513        }
5514        {
5515            {
5516                let values = int64_list_builder.values();
5517                values.append_null();
5518                values.append_value(1);
5519            }
5520            int64_list_builder.append(true);
5521        }
5522        {
5523            {
5524                let values = int64_list_builder.values();
5525                values.append_value(4);
5526            }
5527            int64_list_builder.append(true);
5528        }
5529        let int64_list = int64_list_builder.finish();
5530        let mut utf8_list_builder = ListBuilder::new(StringBuilder::new());
5531        {
5532            {
5533                let values = utf8_list_builder.values();
5534                values.append_value("abc");
5535                values.append_value("efg");
5536                values.append_value("hij");
5537            }
5538            utf8_list_builder.append(true);
5539        }
5540        {
5541            utf8_list_builder.append(false);
5542        }
5543        {
5544            {
5545                let values = utf8_list_builder.values();
5546                values.append_value("efg");
5547                values.append_null();
5548                values.append_value("hij");
5549                values.append_value("xyz");
5550            }
5551            utf8_list_builder.append(true);
5552        }
5553        let utf8_list = utf8_list_builder.finish();
5554        let expected = RecordBatch::try_from_iter_with_nullable([
5555            ("int64_list", Arc::new(int64_list) as Arc<dyn Array>, true),
5556            ("utf8_list", Arc::new(utf8_list) as Arc<dyn Array>, true),
5557        ])
5558        .unwrap();
5559        let batch = read_file(&file, 8, false);
5560        assert_eq!(batch, expected);
5561    }
5562
5563    #[test]
5564    #[cfg(feature = "snappy")]
5565    fn test_nested_lists() {
5566        use arrow_data::ArrayDataBuilder;
5567        let file = arrow_test_data("avro/nested_lists.snappy.avro");
5568        let inner_values = StringArray::from(vec![
5569            Some("a"),
5570            Some("b"),
5571            Some("c"),
5572            Some("d"),
5573            Some("a"),
5574            Some("b"),
5575            Some("c"),
5576            Some("d"),
5577            Some("e"),
5578            Some("a"),
5579            Some("b"),
5580            Some("c"),
5581            Some("d"),
5582            Some("e"),
5583            Some("f"),
5584        ]);
5585        let inner_offsets = Buffer::from_slice_ref([0, 2, 3, 3, 4, 6, 8, 8, 9, 11, 13, 14, 14, 15]);
5586        let inner_validity = [
5587            true, true, false, true, true, true, false, true, true, true, true, false, true,
5588        ];
5589        let inner_null_buffer = Buffer::from_iter(inner_validity.iter().copied());
5590        let inner_field = Field::new("item", DataType::Utf8, true);
5591        let inner_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(inner_field)))
5592            .len(13)
5593            .add_buffer(inner_offsets)
5594            .add_child_data(inner_values.to_data())
5595            .null_bit_buffer(Some(inner_null_buffer))
5596            .build()
5597            .unwrap();
5598        let inner_list_array = ListArray::from(inner_list_data);
5599        let middle_offsets = Buffer::from_slice_ref([0, 2, 4, 6, 8, 11, 13]);
5600        let middle_validity = [true; 6];
5601        let middle_null_buffer = Buffer::from_iter(middle_validity.iter().copied());
5602        let middle_field = Field::new("item", inner_list_array.data_type().clone(), true);
5603        let middle_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(middle_field)))
5604            .len(6)
5605            .add_buffer(middle_offsets)
5606            .add_child_data(inner_list_array.to_data())
5607            .null_bit_buffer(Some(middle_null_buffer))
5608            .build()
5609            .unwrap();
5610        let middle_list_array = ListArray::from(middle_list_data);
5611        let outer_offsets = Buffer::from_slice_ref([0, 2, 4, 6]);
5612        let outer_null_buffer = Buffer::from_slice_ref([0b111]); // all 3 rows valid
5613        let outer_field = Field::new("item", middle_list_array.data_type().clone(), true);
5614        let outer_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(outer_field)))
5615            .len(3)
5616            .add_buffer(outer_offsets)
5617            .add_child_data(middle_list_array.to_data())
5618            .null_bit_buffer(Some(outer_null_buffer))
5619            .build()
5620            .unwrap();
5621        let a_expected = ListArray::from(outer_list_data);
5622        let b_expected = Int32Array::from(vec![1, 1, 1]);
5623        let expected = RecordBatch::try_from_iter_with_nullable([
5624            ("a", Arc::new(a_expected) as Arc<dyn Array>, true),
5625            ("b", Arc::new(b_expected) as Arc<dyn Array>, true),
5626        ])
5627        .unwrap();
5628        let left = read_file(&file, 8, false);
5629        assert_eq!(left, expected, "Mismatch for batch size=8");
5630        let left_small = read_file(&file, 3, false);
5631        assert_eq!(left_small, expected, "Mismatch for batch size=3");
5632    }
5633
5634    #[test]
5635    fn test_simple() {
5636        let tests = [
5637            ("avro/simple_enum.avro", 4, build_expected_enum(), 2),
5638            ("avro/simple_fixed.avro", 2, build_expected_fixed(), 1),
5639        ];
5640
5641        fn build_expected_enum() -> RecordBatch {
5642            // Build the DictionaryArrays for f1, f2, f3
5643            let keys_f1 = Int32Array::from(vec![0, 1, 2, 3]);
5644            let vals_f1 = StringArray::from(vec!["a", "b", "c", "d"]);
5645            let f1_dict =
5646                DictionaryArray::<Int32Type>::try_new(keys_f1, Arc::new(vals_f1)).unwrap();
5647            let keys_f2 = Int32Array::from(vec![2, 3, 0, 1]);
5648            let vals_f2 = StringArray::from(vec!["e", "f", "g", "h"]);
5649            let f2_dict =
5650                DictionaryArray::<Int32Type>::try_new(keys_f2, Arc::new(vals_f2)).unwrap();
5651            let keys_f3 = Int32Array::from(vec![Some(1), Some(2), None, Some(0)]);
5652            let vals_f3 = StringArray::from(vec!["i", "j", "k"]);
5653            let f3_dict =
5654                DictionaryArray::<Int32Type>::try_new(keys_f3, Arc::new(vals_f3)).unwrap();
5655            let dict_type =
5656                DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8));
5657            let mut md_f1 = HashMap::new();
5658            md_f1.insert(
5659                AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
5660                r#"["a","b","c","d"]"#.to_string(),
5661            );
5662            md_f1.insert(AVRO_NAME_METADATA_KEY.to_string(), "enum1".to_string());
5663            md_f1.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns1".to_string());
5664            let f1_field = Field::new("f1", dict_type.clone(), false).with_metadata(md_f1);
5665            let mut md_f2 = HashMap::new();
5666            md_f2.insert(
5667                AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
5668                r#"["e","f","g","h"]"#.to_string(),
5669            );
5670            md_f2.insert(AVRO_NAME_METADATA_KEY.to_string(), "enum2".to_string());
5671            md_f2.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns2".to_string());
5672            let f2_field = Field::new("f2", dict_type.clone(), false).with_metadata(md_f2);
5673            let mut md_f3 = HashMap::new();
5674            md_f3.insert(
5675                AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
5676                r#"["i","j","k"]"#.to_string(),
5677            );
5678            md_f3.insert(AVRO_NAME_METADATA_KEY.to_string(), "enum3".to_string());
5679            md_f3.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns1".to_string());
5680            let f3_field = Field::new("f3", dict_type.clone(), true).with_metadata(md_f3);
5681            let expected_schema = Arc::new(Schema::new(vec![f1_field, f2_field, f3_field]));
5682            RecordBatch::try_new(
5683                expected_schema,
5684                vec![
5685                    Arc::new(f1_dict) as Arc<dyn Array>,
5686                    Arc::new(f2_dict) as Arc<dyn Array>,
5687                    Arc::new(f3_dict) as Arc<dyn Array>,
5688                ],
5689            )
5690            .unwrap()
5691        }
5692
5693        fn build_expected_fixed() -> RecordBatch {
5694            let f1 =
5695                FixedSizeBinaryArray::try_from_iter(vec![b"abcde", b"12345"].into_iter()).unwrap();
5696            let f2 =
5697                FixedSizeBinaryArray::try_from_iter(vec![b"fghijklmno", b"1234567890"].into_iter())
5698                    .unwrap();
5699            let f3 = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
5700                vec![Some(b"ABCDEF" as &[u8]), None].into_iter(),
5701                6,
5702            )
5703            .unwrap();
5704
5705            // Add Avro named-type metadata for fixed fields
5706            let mut md_f1 = HashMap::new();
5707            md_f1.insert(
5708                crate::schema::AVRO_NAME_METADATA_KEY.to_string(),
5709                "fixed1".to_string(),
5710            );
5711            md_f1.insert(
5712                crate::schema::AVRO_NAMESPACE_METADATA_KEY.to_string(),
5713                "ns1".to_string(),
5714            );
5715
5716            let mut md_f2 = HashMap::new();
5717            md_f2.insert(
5718                crate::schema::AVRO_NAME_METADATA_KEY.to_string(),
5719                "fixed2".to_string(),
5720            );
5721            md_f2.insert(
5722                crate::schema::AVRO_NAMESPACE_METADATA_KEY.to_string(),
5723                "ns2".to_string(),
5724            );
5725
5726            let mut md_f3 = HashMap::new();
5727            md_f3.insert(
5728                crate::schema::AVRO_NAME_METADATA_KEY.to_string(),
5729                "fixed3".to_string(),
5730            );
5731            md_f3.insert(
5732                crate::schema::AVRO_NAMESPACE_METADATA_KEY.to_string(),
5733                "ns1".to_string(),
5734            );
5735
5736            let expected_schema = Arc::new(Schema::new(vec![
5737                Field::new("f1", DataType::FixedSizeBinary(5), false).with_metadata(md_f1),
5738                Field::new("f2", DataType::FixedSizeBinary(10), false).with_metadata(md_f2),
5739                Field::new("f3", DataType::FixedSizeBinary(6), true).with_metadata(md_f3),
5740            ]));
5741
5742            RecordBatch::try_new(
5743                expected_schema,
5744                vec![
5745                    Arc::new(f1) as Arc<dyn Array>,
5746                    Arc::new(f2) as Arc<dyn Array>,
5747                    Arc::new(f3) as Arc<dyn Array>,
5748                ],
5749            )
5750            .unwrap()
5751        }
5752        for (file_name, batch_size, expected, alt_batch_size) in tests {
5753            let file = arrow_test_data(file_name);
5754            let actual = read_file(&file, batch_size, false);
5755            assert_eq!(actual, expected);
5756            let actual2 = read_file(&file, alt_batch_size, false);
5757            assert_eq!(actual2, expected);
5758        }
5759    }
5760
5761    #[test]
5762    #[cfg(feature = "snappy")]
5763    fn test_single_nan() {
5764        let file = arrow_test_data("avro/single_nan.avro");
5765        let actual = read_file(&file, 1, false);
5766        use arrow_array::Float64Array;
5767        let schema = Arc::new(Schema::new(vec![Field::new(
5768            "mycol",
5769            DataType::Float64,
5770            true,
5771        )]));
5772        let col = Float64Array::from(vec![None]);
5773        let expected = RecordBatch::try_new(schema, vec![Arc::new(col)]).unwrap();
5774        assert_eq!(actual, expected);
5775        let actual2 = read_file(&file, 2, false);
5776        assert_eq!(actual2, expected);
5777    }
5778
5779    #[test]
5780    fn test_duration_uuid() {
5781        let batch = read_file("test/data/duration_uuid.avro", 4, false);
5782        let schema = batch.schema();
5783        let fields = schema.fields();
5784        assert_eq!(fields.len(), 2);
5785        assert_eq!(fields[0].name(), "duration_field");
5786        assert_eq!(
5787            fields[0].data_type(),
5788            &DataType::Interval(IntervalUnit::MonthDayNano)
5789        );
5790        assert_eq!(fields[1].name(), "uuid_field");
5791        assert_eq!(fields[1].data_type(), &DataType::FixedSizeBinary(16));
5792        assert_eq!(batch.num_rows(), 4);
5793        assert_eq!(batch.num_columns(), 2);
5794        let duration_array = batch
5795            .column(0)
5796            .as_any()
5797            .downcast_ref::<IntervalMonthDayNanoArray>()
5798            .unwrap();
5799        let expected_duration_array: IntervalMonthDayNanoArray = [
5800            Some(IntervalMonthDayNanoType::make_value(1, 15, 500_000_000)),
5801            Some(IntervalMonthDayNanoType::make_value(0, 5, 2_500_000_000)),
5802            Some(IntervalMonthDayNanoType::make_value(2, 0, 0)),
5803            Some(IntervalMonthDayNanoType::make_value(12, 31, 999_000_000)),
5804        ]
5805        .iter()
5806        .copied()
5807        .collect();
5808        assert_eq!(&expected_duration_array, duration_array);
5809        let uuid_array = batch
5810            .column(1)
5811            .as_any()
5812            .downcast_ref::<FixedSizeBinaryArray>()
5813            .unwrap();
5814        let expected_uuid_array = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
5815            [
5816                Some([
5817                    0xfe, 0x7b, 0xc3, 0x0b, 0x4c, 0xe8, 0x4c, 0x5e, 0xb6, 0x7c, 0x22, 0x34, 0xa2,
5818                    0xd3, 0x8e, 0x66,
5819                ]),
5820                Some([
5821                    0xb3, 0x3f, 0x2a, 0xd7, 0x97, 0xb4, 0x4d, 0xe1, 0x8b, 0xfe, 0x94, 0x94, 0x1d,
5822                    0x60, 0x15, 0x6e,
5823                ]),
5824                Some([
5825                    0x5f, 0x74, 0x92, 0x64, 0x07, 0x4b, 0x40, 0x05, 0x84, 0xbf, 0x11, 0x5e, 0xa8,
5826                    0x4e, 0xd2, 0x0a,
5827                ]),
5828                Some([
5829                    0x08, 0x26, 0xcc, 0x06, 0xd2, 0xe3, 0x45, 0x99, 0xb4, 0xad, 0xaf, 0x5f, 0xa6,
5830                    0x90, 0x5c, 0xdb,
5831                ]),
5832            ]
5833            .into_iter(),
5834            16,
5835        )
5836        .unwrap();
5837        assert_eq!(&expected_uuid_array, uuid_array);
5838    }
5839
5840    #[test]
5841    #[cfg(feature = "snappy")]
5842    fn test_datapage_v2() {
5843        let file = arrow_test_data("avro/datapage_v2.snappy.avro");
5844        let batch = read_file(&file, 8, false);
5845        let a = StringArray::from(vec![
5846            Some("abc"),
5847            Some("abc"),
5848            Some("abc"),
5849            None,
5850            Some("abc"),
5851        ]);
5852        let b = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4), Some(5)]);
5853        let c = Float64Array::from(vec![Some(2.0), Some(3.0), Some(4.0), Some(5.0), Some(2.0)]);
5854        let d = BooleanArray::from(vec![
5855            Some(true),
5856            Some(true),
5857            Some(true),
5858            Some(false),
5859            Some(true),
5860        ]);
5861        let e_values = Int32Array::from(vec![
5862            Some(1),
5863            Some(2),
5864            Some(3),
5865            Some(1),
5866            Some(2),
5867            Some(3),
5868            Some(1),
5869            Some(2),
5870        ]);
5871        let e_offsets = OffsetBuffer::new(ScalarBuffer::from(vec![0i32, 3, 3, 3, 6, 8]));
5872        let e_validity = Some(NullBuffer::from(vec![true, false, false, true, true]));
5873        let field_e = Arc::new(Field::new("item", DataType::Int32, true));
5874        let e = ListArray::new(field_e, e_offsets, Arc::new(e_values), e_validity);
5875        let expected = RecordBatch::try_from_iter_with_nullable([
5876            ("a", Arc::new(a) as Arc<dyn Array>, true),
5877            ("b", Arc::new(b) as Arc<dyn Array>, true),
5878            ("c", Arc::new(c) as Arc<dyn Array>, true),
5879            ("d", Arc::new(d) as Arc<dyn Array>, true),
5880            ("e", Arc::new(e) as Arc<dyn Array>, true),
5881        ])
5882        .unwrap();
5883        assert_eq!(batch, expected);
5884    }
5885
5886    #[test]
5887    fn test_nested_records() {
5888        let f1_f1_1 = StringArray::from(vec!["aaa", "bbb"]);
5889        let f1_f1_2 = Int32Array::from(vec![10, 20]);
5890        let rounded_pi = (std::f64::consts::PI * 100.0).round() / 100.0;
5891        let f1_f1_3_1 = Float64Array::from(vec![rounded_pi, rounded_pi]);
5892        let f1_f1_3 = StructArray::from(vec![(
5893            Arc::new(Field::new("f1_3_1", DataType::Float64, false)),
5894            Arc::new(f1_f1_3_1) as Arc<dyn Array>,
5895        )]);
5896        // Add Avro named-type metadata to nested field f1_3 (ns3.record3)
5897        let mut f1_3_md: HashMap<String, String> = HashMap::new();
5898        f1_3_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns3".to_string());
5899        f1_3_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record3".to_string());
5900        let f1_expected = StructArray::from(vec![
5901            (
5902                Arc::new(Field::new("f1_1", DataType::Utf8, false)),
5903                Arc::new(f1_f1_1) as Arc<dyn Array>,
5904            ),
5905            (
5906                Arc::new(Field::new("f1_2", DataType::Int32, false)),
5907                Arc::new(f1_f1_2) as Arc<dyn Array>,
5908            ),
5909            (
5910                Arc::new(
5911                    Field::new(
5912                        "f1_3",
5913                        DataType::Struct(Fields::from(vec![Field::new(
5914                            "f1_3_1",
5915                            DataType::Float64,
5916                            false,
5917                        )])),
5918                        false,
5919                    )
5920                    .with_metadata(f1_3_md),
5921                ),
5922                Arc::new(f1_f1_3) as Arc<dyn Array>,
5923            ),
5924        ]);
5925        let f2_fields = [
5926            Field::new("f2_1", DataType::Boolean, false),
5927            Field::new("f2_2", DataType::Float32, false),
5928        ];
5929        let f2_struct_builder = StructBuilder::new(
5930            f2_fields
5931                .iter()
5932                .map(|f| Arc::new(f.clone()))
5933                .collect::<Vec<Arc<Field>>>(),
5934            vec![
5935                Box::new(BooleanBuilder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>,
5936                Box::new(Float32Builder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>,
5937            ],
5938        );
5939        let mut f2_list_builder = ListBuilder::new(f2_struct_builder);
5940        {
5941            let struct_builder = f2_list_builder.values();
5942            struct_builder.append(true);
5943            {
5944                let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap();
5945                b.append_value(true);
5946            }
5947            {
5948                let b = struct_builder.field_builder::<Float32Builder>(1).unwrap();
5949                b.append_value(1.2_f32);
5950            }
5951            struct_builder.append(true);
5952            {
5953                let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap();
5954                b.append_value(true);
5955            }
5956            {
5957                let b = struct_builder.field_builder::<Float32Builder>(1).unwrap();
5958                b.append_value(2.2_f32);
5959            }
5960            f2_list_builder.append(true);
5961        }
5962        {
5963            let struct_builder = f2_list_builder.values();
5964            struct_builder.append(true);
5965            {
5966                let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap();
5967                b.append_value(false);
5968            }
5969            {
5970                let b = struct_builder.field_builder::<Float32Builder>(1).unwrap();
5971                b.append_value(10.2_f32);
5972            }
5973            f2_list_builder.append(true);
5974        }
5975
5976        let list_array_with_nullable_items = f2_list_builder.finish();
5977        // Add Avro named-type metadata to f2's list item (ns4.record4)
5978        let mut f2_item_md: HashMap<String, String> = HashMap::new();
5979        f2_item_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record4".to_string());
5980        f2_item_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns4".to_string());
5981        let item_field = Arc::new(
5982            Field::new(
5983                "item",
5984                list_array_with_nullable_items.values().data_type().clone(),
5985                false, // items are non-nullable for f2
5986            )
5987            .with_metadata(f2_item_md),
5988        );
5989        let list_data_type = DataType::List(item_field);
5990        let f2_array_data = list_array_with_nullable_items
5991            .to_data()
5992            .into_builder()
5993            .data_type(list_data_type)
5994            .build()
5995            .unwrap();
5996        let f2_expected = ListArray::from(f2_array_data);
5997        let mut f3_struct_builder = StructBuilder::new(
5998            vec![Arc::new(Field::new("f3_1", DataType::Utf8, false))],
5999            vec![Box::new(StringBuilder::new()) as Box<dyn ArrayBuilder>],
6000        );
6001        f3_struct_builder.append(true);
6002        {
6003            let b = f3_struct_builder.field_builder::<StringBuilder>(0).unwrap();
6004            b.append_value("xyz");
6005        }
6006        f3_struct_builder.append(false);
6007        {
6008            let b = f3_struct_builder.field_builder::<StringBuilder>(0).unwrap();
6009            b.append_null();
6010        }
6011        let f3_expected = f3_struct_builder.finish();
6012        let f4_fields = [Field::new("f4_1", DataType::Int64, false)];
6013        let f4_struct_builder = StructBuilder::new(
6014            f4_fields
6015                .iter()
6016                .map(|f| Arc::new(f.clone()))
6017                .collect::<Vec<Arc<Field>>>(),
6018            vec![Box::new(Int64Builder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>],
6019        );
6020        let mut f4_list_builder = ListBuilder::new(f4_struct_builder);
6021        {
6022            let struct_builder = f4_list_builder.values();
6023            struct_builder.append(true);
6024            {
6025                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6026                b.append_value(200);
6027            }
6028            struct_builder.append(false);
6029            {
6030                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6031                b.append_null();
6032            }
6033            f4_list_builder.append(true);
6034        }
6035        {
6036            let struct_builder = f4_list_builder.values();
6037            struct_builder.append(false);
6038            {
6039                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6040                b.append_null();
6041            }
6042            struct_builder.append(true);
6043            {
6044                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6045                b.append_value(300);
6046            }
6047            f4_list_builder.append(true);
6048        }
6049        let f4_expected = f4_list_builder.finish();
6050        // Add Avro named-type metadata to f4's list item (ns6.record6), item is nullable
6051        let mut f4_item_md: HashMap<String, String> = HashMap::new();
6052        f4_item_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns6".to_string());
6053        f4_item_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record6".to_string());
6054        let f4_item_field = Arc::new(
6055            Field::new("item", f4_expected.values().data_type().clone(), true)
6056                .with_metadata(f4_item_md),
6057        );
6058        let f4_list_data_type = DataType::List(f4_item_field);
6059        let f4_array_data = f4_expected
6060            .to_data()
6061            .into_builder()
6062            .data_type(f4_list_data_type)
6063            .build()
6064            .unwrap();
6065        let f4_expected = ListArray::from(f4_array_data);
6066        // Build Schema with Avro named-type metadata on the top-level f1 and f3 fields
6067        let mut f1_md: HashMap<String, String> = HashMap::new();
6068        f1_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record2".to_string());
6069        f1_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns2".to_string());
6070        let mut f3_md: HashMap<String, String> = HashMap::new();
6071        f3_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns5".to_string());
6072        f3_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record5".to_string());
6073        let expected_schema = Schema::new(vec![
6074            Field::new("f1", f1_expected.data_type().clone(), false).with_metadata(f1_md),
6075            Field::new("f2", f2_expected.data_type().clone(), false),
6076            Field::new("f3", f3_expected.data_type().clone(), true).with_metadata(f3_md),
6077            Field::new("f4", f4_expected.data_type().clone(), false),
6078        ]);
6079        let expected = RecordBatch::try_new(
6080            Arc::new(expected_schema),
6081            vec![
6082                Arc::new(f1_expected) as Arc<dyn Array>,
6083                Arc::new(f2_expected) as Arc<dyn Array>,
6084                Arc::new(f3_expected) as Arc<dyn Array>,
6085                Arc::new(f4_expected) as Arc<dyn Array>,
6086            ],
6087        )
6088        .unwrap();
6089        let file = arrow_test_data("avro/nested_records.avro");
6090        let batch_large = read_file(&file, 8, false);
6091        assert_eq!(
6092            batch_large, expected,
6093            "Decoded RecordBatch does not match expected data for nested records (batch size 8)"
6094        );
6095        let batch_small = read_file(&file, 3, false);
6096        assert_eq!(
6097            batch_small, expected,
6098            "Decoded RecordBatch does not match expected data for nested records (batch size 3)"
6099        );
6100    }
6101
6102    #[test]
6103    // TODO: avoid requiring snappy for this file
6104    #[cfg(feature = "snappy")]
6105    fn test_repeated_no_annotation() {
6106        use arrow_data::ArrayDataBuilder;
6107        let file = arrow_test_data("avro/repeated_no_annotation.avro");
6108        let batch_large = read_file(&file, 8, false);
6109        // id column
6110        let id_array = Int32Array::from(vec![1, 2, 3, 4, 5, 6]);
6111        // Build the inner Struct<number:int64, kind:utf8>
6112        let number_array = Int64Array::from(vec![
6113            Some(5555555555),
6114            Some(1111111111),
6115            Some(1111111111),
6116            Some(2222222222),
6117            Some(3333333333),
6118        ]);
6119        let kind_array =
6120            StringArray::from(vec![None, Some("home"), Some("home"), None, Some("mobile")]);
6121        let phone_fields = Fields::from(vec![
6122            Field::new("number", DataType::Int64, true),
6123            Field::new("kind", DataType::Utf8, true),
6124        ]);
6125        let phone_struct_data = ArrayDataBuilder::new(DataType::Struct(phone_fields))
6126            .len(5)
6127            .child_data(vec![number_array.into_data(), kind_array.into_data()])
6128            .build()
6129            .unwrap();
6130        let phone_struct_array = StructArray::from(phone_struct_data);
6131        // Build List<item: Struct<...>> with Avro named-type metadata on the *element* field
6132        let phone_list_offsets = Buffer::from_slice_ref([0i32, 0, 0, 0, 1, 2, 5]);
6133        let phone_list_validity = Buffer::from_iter([false, false, true, true, true, true]);
6134        // The Avro schema names this inner record "phone" in namespace "topLevelRecord.phoneNumbers"
6135        let mut phone_item_md = HashMap::new();
6136        phone_item_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "phone".to_string());
6137        phone_item_md.insert(
6138            AVRO_NAMESPACE_METADATA_KEY.to_string(),
6139            "topLevelRecord.phoneNumbers".to_string(),
6140        );
6141        let phone_item_field = Field::new("item", phone_struct_array.data_type().clone(), true)
6142            .with_metadata(phone_item_md);
6143        let phone_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(phone_item_field)))
6144            .len(6)
6145            .add_buffer(phone_list_offsets)
6146            .null_bit_buffer(Some(phone_list_validity))
6147            .child_data(vec![phone_struct_array.into_data()])
6148            .build()
6149            .unwrap();
6150        let phone_list_array = ListArray::from(phone_list_data);
6151        // Wrap in Struct { phone: List<...> }
6152        let phone_numbers_validity = Buffer::from_iter([false, false, true, true, true, true]);
6153        let phone_numbers_field = Field::new("phone", phone_list_array.data_type().clone(), true);
6154        let phone_numbers_struct_data =
6155            ArrayDataBuilder::new(DataType::Struct(Fields::from(vec![phone_numbers_field])))
6156                .len(6)
6157                .null_bit_buffer(Some(phone_numbers_validity))
6158                .child_data(vec![phone_list_array.into_data()])
6159                .build()
6160                .unwrap();
6161        let phone_numbers_struct_array = StructArray::from(phone_numbers_struct_data);
6162        // Build the expected Schema, annotating the top-level "phoneNumbers" field with Avro name/namespace
6163        let mut phone_numbers_md = HashMap::new();
6164        phone_numbers_md.insert(
6165            AVRO_NAME_METADATA_KEY.to_string(),
6166            "phoneNumbers".to_string(),
6167        );
6168        phone_numbers_md.insert(
6169            AVRO_NAMESPACE_METADATA_KEY.to_string(),
6170            "topLevelRecord".to_string(),
6171        );
6172        let id_field = Field::new("id", DataType::Int32, true);
6173        let phone_numbers_schema_field = Field::new(
6174            "phoneNumbers",
6175            phone_numbers_struct_array.data_type().clone(),
6176            true,
6177        )
6178        .with_metadata(phone_numbers_md);
6179        let expected_schema = Schema::new(vec![id_field, phone_numbers_schema_field]);
6180        // Final expected RecordBatch (arrays already carry matching list-element metadata)
6181        let expected = RecordBatch::try_new(
6182            Arc::new(expected_schema),
6183            vec![
6184                Arc::new(id_array) as _,
6185                Arc::new(phone_numbers_struct_array) as _,
6186            ],
6187        )
6188        .unwrap();
6189        assert_eq!(batch_large, expected, "Mismatch for batch_size=8");
6190        let batch_small = read_file(&file, 3, false);
6191        assert_eq!(batch_small, expected, "Mismatch for batch_size=3");
6192    }
6193
6194    #[test]
6195    // TODO: avoid requiring snappy for this file
6196    #[cfg(feature = "snappy")]
6197    fn test_nonnullable_impala() {
6198        let file = arrow_test_data("avro/nonnullable.impala.avro");
6199        let id = Int64Array::from(vec![Some(8)]);
6200        let mut int_array_builder = ListBuilder::new(Int32Builder::new());
6201        {
6202            let vb = int_array_builder.values();
6203            vb.append_value(-1);
6204        }
6205        int_array_builder.append(true); // finalize one sub-list
6206        let int_array = int_array_builder.finish();
6207        let mut iaa_builder = ListBuilder::new(ListBuilder::new(Int32Builder::new()));
6208        {
6209            let inner_list_builder = iaa_builder.values();
6210            {
6211                let vb = inner_list_builder.values();
6212                vb.append_value(-1);
6213                vb.append_value(-2);
6214            }
6215            inner_list_builder.append(true);
6216            inner_list_builder.append(true);
6217        }
6218        iaa_builder.append(true);
6219        let int_array_array = iaa_builder.finish();
6220        let field_names = MapFieldNames {
6221            entry: "entries".to_string(),
6222            key: "key".to_string(),
6223            value: "value".to_string(),
6224        };
6225        let mut int_map_builder =
6226            MapBuilder::new(Some(field_names), StringBuilder::new(), Int32Builder::new());
6227        {
6228            let (keys, vals) = int_map_builder.entries();
6229            keys.append_value("k1");
6230            vals.append_value(-1);
6231        }
6232        int_map_builder.append(true).unwrap(); // finalize map for row 0
6233        let int_map = int_map_builder.finish();
6234        let field_names2 = MapFieldNames {
6235            entry: "entries".to_string(),
6236            key: "key".to_string(),
6237            value: "value".to_string(),
6238        };
6239        let mut ima_builder = ListBuilder::new(MapBuilder::new(
6240            Some(field_names2),
6241            StringBuilder::new(),
6242            Int32Builder::new(),
6243        ));
6244        {
6245            let map_builder = ima_builder.values();
6246            map_builder.append(true).unwrap();
6247            {
6248                let (keys, vals) = map_builder.entries();
6249                keys.append_value("k1");
6250                vals.append_value(1);
6251            }
6252            map_builder.append(true).unwrap();
6253            map_builder.append(true).unwrap();
6254            map_builder.append(true).unwrap();
6255        }
6256        ima_builder.append(true);
6257        let int_map_array_ = ima_builder.finish();
6258        // Helper metadata maps
6259        let meta_nested_struct: HashMap<String, String> = [
6260            ("avro.name", "nested_Struct"),
6261            ("avro.namespace", "topLevelRecord"),
6262        ]
6263        .into_iter()
6264        .map(|(k, v)| (k.to_string(), v.to_string()))
6265        .collect();
6266        let meta_c: HashMap<String, String> = [
6267            ("avro.name", "c"),
6268            ("avro.namespace", "topLevelRecord.nested_Struct"),
6269        ]
6270        .into_iter()
6271        .map(|(k, v)| (k.to_string(), v.to_string()))
6272        .collect();
6273        let meta_d_item_struct: HashMap<String, String> = [
6274            ("avro.name", "D"),
6275            ("avro.namespace", "topLevelRecord.nested_Struct.c"),
6276        ]
6277        .into_iter()
6278        .map(|(k, v)| (k.to_string(), v.to_string()))
6279        .collect();
6280        let meta_g_value: HashMap<String, String> = [
6281            ("avro.name", "G"),
6282            ("avro.namespace", "topLevelRecord.nested_Struct"),
6283        ]
6284        .into_iter()
6285        .map(|(k, v)| (k.to_string(), v.to_string()))
6286        .collect();
6287        let meta_h: HashMap<String, String> = [
6288            ("avro.name", "h"),
6289            ("avro.namespace", "topLevelRecord.nested_Struct.G"),
6290        ]
6291        .into_iter()
6292        .map(|(k, v)| (k.to_string(), v.to_string()))
6293        .collect();
6294        // Types used multiple times below
6295        let ef_struct_field = Arc::new(
6296            Field::new(
6297                "item",
6298                DataType::Struct(
6299                    vec![
6300                        Field::new("e", DataType::Int32, true),
6301                        Field::new("f", DataType::Utf8, true),
6302                    ]
6303                    .into(),
6304                ),
6305                true,
6306            )
6307            .with_metadata(meta_d_item_struct.clone()),
6308        );
6309        let d_inner_list_field = Arc::new(Field::new(
6310            "item",
6311            DataType::List(ef_struct_field.clone()),
6312            true,
6313        ));
6314        let d_field = Field::new("D", DataType::List(d_inner_list_field.clone()), true);
6315        // G.value.h.i : List<Float64>
6316        let i_list_field = Arc::new(Field::new("item", DataType::Float64, true));
6317        let i_field = Field::new("i", DataType::List(i_list_field.clone()), true);
6318        // G.value.h : Struct<{ i: List<Float64> }> with metadata (h)
6319        let h_field = Field::new("h", DataType::Struct(vec![i_field.clone()].into()), true)
6320            .with_metadata(meta_h.clone());
6321        // G.value : Struct<{ h: ... }> with metadata (G)
6322        let g_value_struct_field = Field::new(
6323            "value",
6324            DataType::Struct(vec![h_field.clone()].into()),
6325            true,
6326        )
6327        .with_metadata(meta_g_value.clone());
6328        // entries struct for Map G
6329        let entries_struct_field = Field::new(
6330            "entries",
6331            DataType::Struct(
6332                vec![
6333                    Field::new("key", DataType::Utf8, false),
6334                    g_value_struct_field.clone(),
6335                ]
6336                .into(),
6337            ),
6338            false,
6339        );
6340        // Top-level nested_Struct fields (include metadata on "c")
6341        let a_field = Arc::new(Field::new("a", DataType::Int32, true));
6342        let b_field = Arc::new(Field::new(
6343            "B",
6344            DataType::List(Arc::new(Field::new("item", DataType::Int32, true))),
6345            true,
6346        ));
6347        let c_field = Arc::new(
6348            Field::new("c", DataType::Struct(vec![d_field.clone()].into()), true)
6349                .with_metadata(meta_c.clone()),
6350        );
6351        let g_field = Arc::new(Field::new(
6352            "G",
6353            DataType::Map(Arc::new(entries_struct_field.clone()), false),
6354            true,
6355        ));
6356        // Now create builders that match these exact field types (so nested types carry metadata)
6357        let mut nested_sb = StructBuilder::new(
6358            vec![
6359                a_field.clone(),
6360                b_field.clone(),
6361                c_field.clone(),
6362                g_field.clone(),
6363            ],
6364            vec![
6365                Box::new(Int32Builder::new()),
6366                Box::new(ListBuilder::new(Int32Builder::new())),
6367                {
6368                    // builder for "c" with correctly typed "D" including metadata on inner list item
6369                    Box::new(StructBuilder::new(
6370                        vec![Arc::new(d_field.clone())],
6371                        vec![Box::new({
6372                            let ef_struct_builder = StructBuilder::new(
6373                                vec![
6374                                    Arc::new(Field::new("e", DataType::Int32, true)),
6375                                    Arc::new(Field::new("f", DataType::Utf8, true)),
6376                                ],
6377                                vec![
6378                                    Box::new(Int32Builder::new()),
6379                                    Box::new(StringBuilder::new()),
6380                                ],
6381                            );
6382                            // Inner list that holds Struct<e,f> with Avro named-type metadata ("D")
6383                            let list_of_ef = ListBuilder::new(ef_struct_builder)
6384                                .with_field(ef_struct_field.clone());
6385                            // Outer list for "D"
6386                            ListBuilder::new(list_of_ef)
6387                        })],
6388                    ))
6389                },
6390                {
6391                    let map_field_names = MapFieldNames {
6392                        entry: "entries".to_string(),
6393                        key: "key".to_string(),
6394                        value: "value".to_string(),
6395                    };
6396                    let i_list_builder = ListBuilder::new(Float64Builder::new());
6397                    let h_struct_builder = StructBuilder::new(
6398                        vec![Arc::new(Field::new(
6399                            "i",
6400                            DataType::List(i_list_field.clone()),
6401                            true,
6402                        ))],
6403                        vec![Box::new(i_list_builder)],
6404                    );
6405                    let g_value_builder = StructBuilder::new(
6406                        vec![Arc::new(
6407                            Field::new("h", DataType::Struct(vec![i_field.clone()].into()), true)
6408                                .with_metadata(meta_h.clone()),
6409                        )],
6410                        vec![Box::new(h_struct_builder)],
6411                    );
6412                    // Use with_values_field to attach metadata to "value" field in the map's entries
6413                    let map_builder = MapBuilder::new(
6414                        Some(map_field_names),
6415                        StringBuilder::new(),
6416                        g_value_builder,
6417                    )
6418                    .with_values_field(Arc::new(
6419                        Field::new(
6420                            "value",
6421                            DataType::Struct(vec![h_field.clone()].into()),
6422                            true,
6423                        )
6424                        .with_metadata(meta_g_value.clone()),
6425                    ));
6426
6427                    Box::new(map_builder)
6428                },
6429            ],
6430        );
6431        nested_sb.append(true);
6432        {
6433            let a_builder = nested_sb.field_builder::<Int32Builder>(0).unwrap();
6434            a_builder.append_value(-1);
6435        }
6436        {
6437            let b_builder = nested_sb
6438                .field_builder::<ListBuilder<Int32Builder>>(1)
6439                .unwrap();
6440            {
6441                let vb = b_builder.values();
6442                vb.append_value(-1);
6443            }
6444            b_builder.append(true);
6445        }
6446        {
6447            let c_struct_builder = nested_sb.field_builder::<StructBuilder>(2).unwrap();
6448            c_struct_builder.append(true);
6449            let d_list_builder = c_struct_builder
6450                .field_builder::<ListBuilder<ListBuilder<StructBuilder>>>(0)
6451                .unwrap();
6452            {
6453                let sub_list_builder = d_list_builder.values();
6454                {
6455                    let ef_struct = sub_list_builder.values();
6456                    ef_struct.append(true);
6457                    {
6458                        let e_b = ef_struct.field_builder::<Int32Builder>(0).unwrap();
6459                        e_b.append_value(-1);
6460                        let f_b = ef_struct.field_builder::<StringBuilder>(1).unwrap();
6461                        f_b.append_value("nonnullable");
6462                    }
6463                    sub_list_builder.append(true);
6464                }
6465                d_list_builder.append(true);
6466            }
6467        }
6468        {
6469            let g_map_builder = nested_sb
6470                .field_builder::<MapBuilder<StringBuilder, StructBuilder>>(3)
6471                .unwrap();
6472            g_map_builder.append(true).unwrap();
6473        }
6474        let nested_struct = nested_sb.finish();
6475        let schema = Arc::new(arrow_schema::Schema::new(vec![
6476            Field::new("ID", id.data_type().clone(), true),
6477            Field::new("Int_Array", int_array.data_type().clone(), true),
6478            Field::new("int_array_array", int_array_array.data_type().clone(), true),
6479            Field::new("Int_Map", int_map.data_type().clone(), true),
6480            Field::new("int_map_array", int_map_array_.data_type().clone(), true),
6481            Field::new("nested_Struct", nested_struct.data_type().clone(), true)
6482                .with_metadata(meta_nested_struct.clone()),
6483        ]));
6484        let expected = RecordBatch::try_new(
6485            schema,
6486            vec![
6487                Arc::new(id) as Arc<dyn Array>,
6488                Arc::new(int_array),
6489                Arc::new(int_array_array),
6490                Arc::new(int_map),
6491                Arc::new(int_map_array_),
6492                Arc::new(nested_struct),
6493            ],
6494        )
6495        .unwrap();
6496        let batch_large = read_file(&file, 8, false);
6497        assert_eq!(batch_large, expected, "Mismatch for batch_size=8");
6498        let batch_small = read_file(&file, 3, false);
6499        assert_eq!(batch_small, expected, "Mismatch for batch_size=3");
6500    }
6501
6502    #[test]
6503    fn test_nonnullable_impala_strict() {
6504        let file = arrow_test_data("avro/nonnullable.impala.avro");
6505        let err = read_file_strict(&file, 8, false).unwrap_err();
6506        assert!(err.to_string().contains(
6507            "Found Avro union of the form ['T','null'], which is disallowed in strict_mode"
6508        ));
6509    }
6510
6511    #[test]
6512    // TODO: avoid requiring snappy for this file
6513    #[cfg(feature = "snappy")]
6514    fn test_nullable_impala() {
6515        let file = arrow_test_data("avro/nullable.impala.avro");
6516        let batch1 = read_file(&file, 3, false);
6517        let batch2 = read_file(&file, 8, false);
6518        assert_eq!(batch1, batch2);
6519        let batch = batch1;
6520        assert_eq!(batch.num_rows(), 7);
6521        let id_array = batch
6522            .column(0)
6523            .as_any()
6524            .downcast_ref::<Int64Array>()
6525            .expect("id column should be an Int64Array");
6526        let expected_ids = [1, 2, 3, 4, 5, 6, 7];
6527        for (i, &expected_id) in expected_ids.iter().enumerate() {
6528            assert_eq!(id_array.value(i), expected_id, "Mismatch in id at row {i}",);
6529        }
6530        let int_array = batch
6531            .column(1)
6532            .as_any()
6533            .downcast_ref::<ListArray>()
6534            .expect("int_array column should be a ListArray");
6535        {
6536            let offsets = int_array.value_offsets();
6537            let start = offsets[0] as usize;
6538            let end = offsets[1] as usize;
6539            let values = int_array
6540                .values()
6541                .as_any()
6542                .downcast_ref::<Int32Array>()
6543                .expect("Values of int_array should be an Int32Array");
6544            let row0: Vec<Option<i32>> = (start..end).map(|i| Some(values.value(i))).collect();
6545            assert_eq!(
6546                row0,
6547                vec![Some(1), Some(2), Some(3)],
6548                "Mismatch in int_array row 0"
6549            );
6550        }
6551        let nested_struct = batch
6552            .column(5)
6553            .as_any()
6554            .downcast_ref::<StructArray>()
6555            .expect("nested_struct column should be a StructArray");
6556        let a_array = nested_struct
6557            .column_by_name("A")
6558            .expect("Field A should exist in nested_struct")
6559            .as_any()
6560            .downcast_ref::<Int32Array>()
6561            .expect("Field A should be an Int32Array");
6562        assert_eq!(a_array.value(0), 1, "Mismatch in nested_struct.A at row 0");
6563        assert!(
6564            !a_array.is_valid(1),
6565            "Expected null in nested_struct.A at row 1"
6566        );
6567        assert!(
6568            !a_array.is_valid(3),
6569            "Expected null in nested_struct.A at row 3"
6570        );
6571        assert_eq!(a_array.value(6), 7, "Mismatch in nested_struct.A at row 6");
6572    }
6573
6574    #[test]
6575    fn test_nullable_impala_strict() {
6576        let file = arrow_test_data("avro/nullable.impala.avro");
6577        let err = read_file_strict(&file, 8, false).unwrap_err();
6578        assert!(err.to_string().contains(
6579            "Found Avro union of the form ['T','null'], which is disallowed in strict_mode"
6580        ));
6581    }
6582
6583    #[test]
6584    fn test_nested_record_type_reuse() {
6585        // The .avro file has the following schema:
6586        // {
6587        // "type" : "record",
6588        // "name" : "Record",
6589        // "fields" : [ {
6590        //     "name" : "nested",
6591        //     "type" : {
6592        //     "type" : "record",
6593        //     "name" : "Nested",
6594        //     "fields" : [ {
6595        //         "name" : "nested_int",
6596        //         "type" : "int"
6597        //     } ]
6598        //     }
6599        // }, {
6600        //     "name" : "nestedRecord",
6601        //     "type" : "Nested"
6602        // }, {
6603        //     "name" : "nestedArray",
6604        //     "type" : {
6605        //     "type" : "array",
6606        //     "items" : "Nested"
6607        //     }
6608        // } ]
6609        // }
6610        let batch = read_file("test/data/nested_record_reuse.avro", 8, false);
6611        let schema = batch.schema();
6612
6613        // Verify schema structure
6614        assert_eq!(schema.fields().len(), 3);
6615        let fields = schema.fields();
6616        assert_eq!(fields[0].name(), "nested");
6617        assert_eq!(fields[1].name(), "nestedRecord");
6618        assert_eq!(fields[2].name(), "nestedArray");
6619        assert!(matches!(fields[0].data_type(), DataType::Struct(_)));
6620        assert!(matches!(fields[1].data_type(), DataType::Struct(_)));
6621        assert!(matches!(fields[2].data_type(), DataType::List(_)));
6622
6623        // Validate that the nested record type
6624        if let DataType::Struct(nested_fields) = fields[0].data_type() {
6625            assert_eq!(nested_fields.len(), 1);
6626            assert_eq!(nested_fields[0].name(), "nested_int");
6627            assert_eq!(nested_fields[0].data_type(), &DataType::Int32);
6628        }
6629
6630        // Validate that the nested record type is reused
6631        assert_eq!(fields[0].data_type(), fields[1].data_type());
6632        if let DataType::List(array_field) = fields[2].data_type() {
6633            assert_eq!(array_field.data_type(), fields[0].data_type());
6634        }
6635
6636        // Validate data
6637        assert_eq!(batch.num_rows(), 2);
6638        assert_eq!(batch.num_columns(), 3);
6639
6640        // Validate the first column (nested)
6641        let nested_col = batch
6642            .column(0)
6643            .as_any()
6644            .downcast_ref::<StructArray>()
6645            .unwrap();
6646        let nested_int_array = nested_col
6647            .column_by_name("nested_int")
6648            .unwrap()
6649            .as_any()
6650            .downcast_ref::<Int32Array>()
6651            .unwrap();
6652        assert_eq!(nested_int_array.value(0), 42);
6653        assert_eq!(nested_int_array.value(1), 99);
6654
6655        // Validate the second column (nestedRecord)
6656        let nested_record_col = batch
6657            .column(1)
6658            .as_any()
6659            .downcast_ref::<StructArray>()
6660            .unwrap();
6661        let nested_record_int_array = nested_record_col
6662            .column_by_name("nested_int")
6663            .unwrap()
6664            .as_any()
6665            .downcast_ref::<Int32Array>()
6666            .unwrap();
6667        assert_eq!(nested_record_int_array.value(0), 100);
6668        assert_eq!(nested_record_int_array.value(1), 200);
6669
6670        // Validate the third column (nestedArray)
6671        let nested_array_col = batch
6672            .column(2)
6673            .as_any()
6674            .downcast_ref::<ListArray>()
6675            .unwrap();
6676        assert_eq!(nested_array_col.len(), 2);
6677        let first_array_struct = nested_array_col.value(0);
6678        let first_array_struct_array = first_array_struct
6679            .as_any()
6680            .downcast_ref::<StructArray>()
6681            .unwrap();
6682        let first_array_int_values = first_array_struct_array
6683            .column_by_name("nested_int")
6684            .unwrap()
6685            .as_any()
6686            .downcast_ref::<Int32Array>()
6687            .unwrap();
6688        assert_eq!(first_array_int_values.len(), 3);
6689        assert_eq!(first_array_int_values.value(0), 1);
6690        assert_eq!(first_array_int_values.value(1), 2);
6691        assert_eq!(first_array_int_values.value(2), 3);
6692    }
6693
6694    #[test]
6695    fn test_enum_type_reuse() {
6696        // The .avro file has the following schema:
6697        // {
6698        //     "type" : "record",
6699        //     "name" : "Record",
6700        //     "fields" : [ {
6701        //       "name" : "status",
6702        //       "type" : {
6703        //         "type" : "enum",
6704        //         "name" : "Status",
6705        //         "symbols" : [ "ACTIVE", "INACTIVE", "PENDING" ]
6706        //       }
6707        //     }, {
6708        //       "name" : "backupStatus",
6709        //       "type" : "Status"
6710        //     }, {
6711        //       "name" : "statusHistory",
6712        //       "type" : {
6713        //         "type" : "array",
6714        //         "items" : "Status"
6715        //       }
6716        //     } ]
6717        //   }
6718        let batch = read_file("test/data/enum_reuse.avro", 8, false);
6719        let schema = batch.schema();
6720
6721        // Verify schema structure
6722        assert_eq!(schema.fields().len(), 3);
6723        let fields = schema.fields();
6724        assert_eq!(fields[0].name(), "status");
6725        assert_eq!(fields[1].name(), "backupStatus");
6726        assert_eq!(fields[2].name(), "statusHistory");
6727        assert!(matches!(fields[0].data_type(), DataType::Dictionary(_, _)));
6728        assert!(matches!(fields[1].data_type(), DataType::Dictionary(_, _)));
6729        assert!(matches!(fields[2].data_type(), DataType::List(_)));
6730
6731        if let DataType::Dictionary(key_type, value_type) = fields[0].data_type() {
6732            assert_eq!(key_type.as_ref(), &DataType::Int32);
6733            assert_eq!(value_type.as_ref(), &DataType::Utf8);
6734        }
6735
6736        // Validate that the enum types are reused
6737        assert_eq!(fields[0].data_type(), fields[1].data_type());
6738        if let DataType::List(array_field) = fields[2].data_type() {
6739            assert_eq!(array_field.data_type(), fields[0].data_type());
6740        }
6741
6742        // Validate data - should have 2 rows
6743        assert_eq!(batch.num_rows(), 2);
6744        assert_eq!(batch.num_columns(), 3);
6745
6746        // Get status enum values
6747        let status_col = batch
6748            .column(0)
6749            .as_any()
6750            .downcast_ref::<DictionaryArray<Int32Type>>()
6751            .unwrap();
6752        let status_values = status_col
6753            .values()
6754            .as_any()
6755            .downcast_ref::<StringArray>()
6756            .unwrap();
6757
6758        // First row should be "ACTIVE", second row should be "PENDING"
6759        assert_eq!(
6760            status_values.value(status_col.key(0).unwrap() as usize),
6761            "ACTIVE"
6762        );
6763        assert_eq!(
6764            status_values.value(status_col.key(1).unwrap() as usize),
6765            "PENDING"
6766        );
6767
6768        // Get backupStatus enum values (same as status)
6769        let backup_status_col = batch
6770            .column(1)
6771            .as_any()
6772            .downcast_ref::<DictionaryArray<Int32Type>>()
6773            .unwrap();
6774        let backup_status_values = backup_status_col
6775            .values()
6776            .as_any()
6777            .downcast_ref::<StringArray>()
6778            .unwrap();
6779
6780        // First row should be "INACTIVE", second row should be "ACTIVE"
6781        assert_eq!(
6782            backup_status_values.value(backup_status_col.key(0).unwrap() as usize),
6783            "INACTIVE"
6784        );
6785        assert_eq!(
6786            backup_status_values.value(backup_status_col.key(1).unwrap() as usize),
6787            "ACTIVE"
6788        );
6789
6790        // Get statusHistory array
6791        let status_history_col = batch
6792            .column(2)
6793            .as_any()
6794            .downcast_ref::<ListArray>()
6795            .unwrap();
6796        assert_eq!(status_history_col.len(), 2);
6797
6798        // Validate first row's array data
6799        let first_array_dict = status_history_col.value(0);
6800        let first_array_dict_array = first_array_dict
6801            .as_any()
6802            .downcast_ref::<DictionaryArray<Int32Type>>()
6803            .unwrap();
6804        let first_array_values = first_array_dict_array
6805            .values()
6806            .as_any()
6807            .downcast_ref::<StringArray>()
6808            .unwrap();
6809
6810        // First row: ["PENDING", "ACTIVE", "INACTIVE"]
6811        assert_eq!(first_array_dict_array.len(), 3);
6812        assert_eq!(
6813            first_array_values.value(first_array_dict_array.key(0).unwrap() as usize),
6814            "PENDING"
6815        );
6816        assert_eq!(
6817            first_array_values.value(first_array_dict_array.key(1).unwrap() as usize),
6818            "ACTIVE"
6819        );
6820        assert_eq!(
6821            first_array_values.value(first_array_dict_array.key(2).unwrap() as usize),
6822            "INACTIVE"
6823        );
6824    }
6825
6826    #[test]
6827    fn comprehensive_e2e_test() {
6828        let path = "test/data/comprehensive_e2e.avro";
6829        let batch = read_file(path, 1024, false);
6830        let schema = batch.schema();
6831
6832        #[inline]
6833        fn tid_by_name(fields: &UnionFields, want: &str) -> i8 {
6834            for (tid, f) in fields.iter() {
6835                if f.name() == want {
6836                    return tid;
6837                }
6838            }
6839            panic!("union child '{want}' not found");
6840        }
6841
6842        #[inline]
6843        fn tid_by_dt(fields: &UnionFields, pred: impl Fn(&DataType) -> bool) -> i8 {
6844            for (tid, f) in fields.iter() {
6845                if pred(f.data_type()) {
6846                    return tid;
6847                }
6848            }
6849            panic!("no union child matches predicate");
6850        }
6851
6852        fn mk_dense_union(
6853            fields: &UnionFields,
6854            type_ids: Vec<i8>,
6855            offsets: Vec<i32>,
6856            provide: impl Fn(&Field) -> Option<ArrayRef>,
6857        ) -> ArrayRef {
6858            fn empty_child_for(dt: &DataType) -> Arc<dyn Array> {
6859                match dt {
6860                    DataType::Null => Arc::new(NullArray::new(0)),
6861                    DataType::Boolean => Arc::new(BooleanArray::from(Vec::<bool>::new())),
6862                    DataType::Int32 => Arc::new(Int32Array::from(Vec::<i32>::new())),
6863                    DataType::Int64 => Arc::new(Int64Array::from(Vec::<i64>::new())),
6864                    DataType::Float32 => Arc::new(Float32Array::from(Vec::<f32>::new())),
6865                    DataType::Float64 => Arc::new(Float64Array::from(Vec::<f64>::new())),
6866                    DataType::Binary => Arc::new(BinaryArray::from(Vec::<&[u8]>::new())),
6867                    DataType::Utf8 => Arc::new(StringArray::from(Vec::<&str>::new())),
6868                    DataType::Date32 => Arc::new(Date32Array::from(Vec::<i32>::new())),
6869                    DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
6870                        Arc::new(Time32MillisecondArray::from(Vec::<i32>::new()))
6871                    }
6872                    DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
6873                        Arc::new(Time64MicrosecondArray::from(Vec::<i64>::new()))
6874                    }
6875                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
6876                        let a = TimestampMillisecondArray::from(Vec::<i64>::new());
6877                        Arc::new(if let Some(tz) = tz {
6878                            a.with_timezone(tz.clone())
6879                        } else {
6880                            a
6881                        })
6882                    }
6883                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
6884                        let a = TimestampMicrosecondArray::from(Vec::<i64>::new());
6885                        Arc::new(if let Some(tz) = tz {
6886                            a.with_timezone(tz.clone())
6887                        } else {
6888                            a
6889                        })
6890                    }
6891                    DataType::Interval(IntervalUnit::MonthDayNano) => Arc::new(
6892                        IntervalMonthDayNanoArray::from(Vec::<IntervalMonthDayNano>::new()),
6893                    ),
6894                    DataType::FixedSizeBinary(sz) => Arc::new(
6895                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(
6896                            std::iter::empty::<Option<Vec<u8>>>(),
6897                            *sz,
6898                        )
6899                        .unwrap(),
6900                    ),
6901                    DataType::Dictionary(_, _) => {
6902                        let keys = Int32Array::from(Vec::<i32>::new());
6903                        let values = Arc::new(StringArray::from(Vec::<&str>::new()));
6904                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
6905                    }
6906                    DataType::Struct(fields) => {
6907                        let children: Vec<ArrayRef> = fields
6908                            .iter()
6909                            .map(|f| empty_child_for(f.data_type()) as ArrayRef)
6910                            .collect();
6911                        Arc::new(StructArray::new(fields.clone(), children, None))
6912                    }
6913                    DataType::List(field) => {
6914                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
6915                        Arc::new(
6916                            ListArray::try_new(
6917                                field.clone(),
6918                                offsets,
6919                                empty_child_for(field.data_type()),
6920                                None,
6921                            )
6922                            .unwrap(),
6923                        )
6924                    }
6925                    DataType::Map(entry_field, is_sorted) => {
6926                        let (key_field, val_field) = match entry_field.data_type() {
6927                            DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
6928                            other => panic!("unexpected map entries type: {other:?}"),
6929                        };
6930                        let keys = StringArray::from(Vec::<&str>::new());
6931                        let vals: ArrayRef = match val_field.data_type() {
6932                            DataType::Null => Arc::new(NullArray::new(0)) as ArrayRef,
6933                            DataType::Boolean => {
6934                                Arc::new(BooleanArray::from(Vec::<bool>::new())) as ArrayRef
6935                            }
6936                            DataType::Int32 => {
6937                                Arc::new(Int32Array::from(Vec::<i32>::new())) as ArrayRef
6938                            }
6939                            DataType::Int64 => {
6940                                Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
6941                            }
6942                            DataType::Float32 => {
6943                                Arc::new(Float32Array::from(Vec::<f32>::new())) as ArrayRef
6944                            }
6945                            DataType::Float64 => {
6946                                Arc::new(Float64Array::from(Vec::<f64>::new())) as ArrayRef
6947                            }
6948                            DataType::Utf8 => {
6949                                Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
6950                            }
6951                            DataType::Binary => {
6952                                Arc::new(BinaryArray::from(Vec::<&[u8]>::new())) as ArrayRef
6953                            }
6954                            DataType::Union(uf, _) => {
6955                                let children: Vec<ArrayRef> = uf
6956                                    .iter()
6957                                    .map(|(_, f)| empty_child_for(f.data_type()))
6958                                    .collect();
6959                                Arc::new(
6960                                    UnionArray::try_new(
6961                                        uf.clone(),
6962                                        ScalarBuffer::<i8>::from(Vec::<i8>::new()),
6963                                        Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
6964                                        children,
6965                                    )
6966                                    .unwrap(),
6967                                ) as ArrayRef
6968                            }
6969                            other => panic!("unsupported map value type: {other:?}"),
6970                        };
6971                        let entries = StructArray::new(
6972                            Fields::from(vec![
6973                                key_field.as_ref().clone(),
6974                                val_field.as_ref().clone(),
6975                            ]),
6976                            vec![Arc::new(keys) as ArrayRef, vals],
6977                            None,
6978                        );
6979                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
6980                        Arc::new(MapArray::new(
6981                            entry_field.clone(),
6982                            offsets,
6983                            entries,
6984                            None,
6985                            *is_sorted,
6986                        ))
6987                    }
6988                    other => panic!("empty_child_for: unhandled type {other:?}"),
6989                }
6990            }
6991            let children: Vec<ArrayRef> = fields
6992                .iter()
6993                .map(|(_, f)| provide(f).unwrap_or_else(|| empty_child_for(f.data_type())))
6994                .collect();
6995            Arc::new(
6996                UnionArray::try_new(
6997                    fields.clone(),
6998                    ScalarBuffer::<i8>::from(type_ids),
6999                    Some(ScalarBuffer::<i32>::from(offsets)),
7000                    children,
7001                )
7002                .unwrap(),
7003            ) as ArrayRef
7004        }
7005
7006        #[inline]
7007        fn uuid16_from_str(s: &str) -> [u8; 16] {
7008            let mut out = [0u8; 16];
7009            let mut idx = 0usize;
7010            let mut hi: Option<u8> = None;
7011            for ch in s.chars() {
7012                if ch == '-' {
7013                    continue;
7014                }
7015                let v = ch.to_digit(16).expect("invalid hex digit in UUID") as u8;
7016                if let Some(h) = hi {
7017                    out[idx] = (h << 4) | v;
7018                    idx += 1;
7019                    hi = None;
7020                } else {
7021                    hi = Some(v);
7022                }
7023            }
7024            assert_eq!(idx, 16, "UUID must decode to 16 bytes");
7025            out
7026        }
7027        let date_a: i32 = 19_000; // 2022-01-08
7028        let time_ms_a: i32 = 12 * 3_600_000 + 34 * 60_000 + 56_000 + 789;
7029        let time_us_eod: i64 = 86_400_000_000 - 1;
7030        let ts_ms_2024_01_01: i64 = 1_704_067_200_000; // 2024-01-01T00:00:00Z
7031        let ts_us_2024_01_01: i64 = ts_ms_2024_01_01 * 1_000;
7032        let dur_small = IntervalMonthDayNanoType::make_value(1, 2, 3_000_000_000);
7033        let dur_zero = IntervalMonthDayNanoType::make_value(0, 0, 0);
7034        let dur_large =
7035            IntervalMonthDayNanoType::make_value(12, 31, ((86_400_000 - 1) as i64) * 1_000_000);
7036        let dur_2years = IntervalMonthDayNanoType::make_value(24, 0, 0);
7037        let uuid1 = uuid16_from_str("fe7bc30b-4ce8-4c5e-b67c-2234a2d38e66");
7038        let uuid2 = uuid16_from_str("0826cc06-d2e3-4599-b4ad-af5fa6905cdb");
7039
7040        #[inline]
7041        fn push_like(
7042            reader_schema: &arrow_schema::Schema,
7043            name: &str,
7044            arr: ArrayRef,
7045            fields: &mut Vec<FieldRef>,
7046            cols: &mut Vec<ArrayRef>,
7047        ) {
7048            let src = reader_schema
7049                .field_with_name(name)
7050                .unwrap_or_else(|_| panic!("source schema missing field '{name}'"));
7051            let mut f = Field::new(name, arr.data_type().clone(), src.is_nullable());
7052            let md = src.metadata();
7053            if !md.is_empty() {
7054                f = f.with_metadata(md.clone());
7055            }
7056            fields.push(Arc::new(f));
7057            cols.push(arr);
7058        }
7059
7060        let mut fields: Vec<FieldRef> = Vec::new();
7061        let mut columns: Vec<ArrayRef> = Vec::new();
7062        push_like(
7063            schema.as_ref(),
7064            "id",
7065            Arc::new(Int64Array::from(vec![1, 2, 3, 4])) as ArrayRef,
7066            &mut fields,
7067            &mut columns,
7068        );
7069        push_like(
7070            schema.as_ref(),
7071            "flag",
7072            Arc::new(BooleanArray::from(vec![true, false, true, false])) as ArrayRef,
7073            &mut fields,
7074            &mut columns,
7075        );
7076        push_like(
7077            schema.as_ref(),
7078            "ratio_f32",
7079            Arc::new(Float32Array::from(vec![1.25f32, -0.0, 3.5, 9.75])) as ArrayRef,
7080            &mut fields,
7081            &mut columns,
7082        );
7083        push_like(
7084            schema.as_ref(),
7085            "ratio_f64",
7086            Arc::new(Float64Array::from(vec![2.5f64, -1.0, 7.0, -2.25])) as ArrayRef,
7087            &mut fields,
7088            &mut columns,
7089        );
7090        push_like(
7091            schema.as_ref(),
7092            "count_i32",
7093            Arc::new(Int32Array::from(vec![7, -1, 0, 123])) as ArrayRef,
7094            &mut fields,
7095            &mut columns,
7096        );
7097        push_like(
7098            schema.as_ref(),
7099            "count_i64",
7100            Arc::new(Int64Array::from(vec![
7101                7_000_000_000i64,
7102                -2,
7103                0,
7104                -9_876_543_210i64,
7105            ])) as ArrayRef,
7106            &mut fields,
7107            &mut columns,
7108        );
7109        push_like(
7110            schema.as_ref(),
7111            "opt_i32_nullfirst",
7112            Arc::new(Int32Array::from(vec![None, Some(42), None, Some(0)])) as ArrayRef,
7113            &mut fields,
7114            &mut columns,
7115        );
7116        push_like(
7117            schema.as_ref(),
7118            "opt_str_nullsecond",
7119            Arc::new(StringArray::from(vec![
7120                Some("alpha"),
7121                None,
7122                Some("s3"),
7123                Some(""),
7124            ])) as ArrayRef,
7125            &mut fields,
7126            &mut columns,
7127        );
7128        {
7129            let uf = match schema
7130                .field_with_name("tri_union_prim")
7131                .unwrap()
7132                .data_type()
7133            {
7134                DataType::Union(f, UnionMode::Dense) => f.clone(),
7135                other => panic!("tri_union_prim should be dense union, got {other:?}"),
7136            };
7137            let tid_i = tid_by_name(&uf, "int");
7138            let tid_s = tid_by_name(&uf, "string");
7139            let tid_b = tid_by_name(&uf, "boolean");
7140            let tids = vec![tid_i, tid_s, tid_b, tid_s];
7141            let offs = vec![0, 0, 0, 1];
7142            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7143                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![0])) as ArrayRef),
7144                DataType::Utf8 => Some(Arc::new(StringArray::from(vec!["hi", ""])) as ArrayRef),
7145                DataType::Boolean => Some(Arc::new(BooleanArray::from(vec![true])) as ArrayRef),
7146                _ => None,
7147            });
7148            push_like(
7149                schema.as_ref(),
7150                "tri_union_prim",
7151                arr,
7152                &mut fields,
7153                &mut columns,
7154            );
7155        }
7156
7157        push_like(
7158            schema.as_ref(),
7159            "str_utf8",
7160            Arc::new(StringArray::from(vec!["hello", "", "world", "✓ unicode"])) as ArrayRef,
7161            &mut fields,
7162            &mut columns,
7163        );
7164        push_like(
7165            schema.as_ref(),
7166            "raw_bytes",
7167            Arc::new(BinaryArray::from(vec![
7168                b"\x00\x01".as_ref(),
7169                b"".as_ref(),
7170                b"\xFF\x00".as_ref(),
7171                b"\x10\x20\x30\x40".as_ref(),
7172            ])) as ArrayRef,
7173            &mut fields,
7174            &mut columns,
7175        );
7176        {
7177            let it = [
7178                Some(*b"0123456789ABCDEF"),
7179                Some([0u8; 16]),
7180                Some(*b"ABCDEFGHIJKLMNOP"),
7181                Some([0xAA; 16]),
7182            ]
7183            .into_iter();
7184            let arr =
7185                Arc::new(FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap())
7186                    as ArrayRef;
7187            push_like(
7188                schema.as_ref(),
7189                "fx16_plain",
7190                arr,
7191                &mut fields,
7192                &mut columns,
7193            );
7194        }
7195        {
7196            #[cfg(feature = "small_decimals")]
7197            let dec10_2 = Arc::new(
7198                Decimal64Array::from_iter_values([123456i64, -1, 0, 9_999_999_999i64])
7199                    .with_precision_and_scale(10, 2)
7200                    .unwrap(),
7201            ) as ArrayRef;
7202            #[cfg(not(feature = "small_decimals"))]
7203            let dec10_2 = Arc::new(
7204                Decimal128Array::from_iter_values([123456i128, -1, 0, 9_999_999_999i128])
7205                    .with_precision_and_scale(10, 2)
7206                    .unwrap(),
7207            ) as ArrayRef;
7208            push_like(
7209                schema.as_ref(),
7210                "dec_bytes_s10_2",
7211                dec10_2,
7212                &mut fields,
7213                &mut columns,
7214            );
7215        }
7216        {
7217            #[cfg(feature = "small_decimals")]
7218            let dec20_4 = Arc::new(
7219                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
7220                    .with_precision_and_scale(20, 4)
7221                    .unwrap(),
7222            ) as ArrayRef;
7223            #[cfg(not(feature = "small_decimals"))]
7224            let dec20_4 = Arc::new(
7225                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
7226                    .with_precision_and_scale(20, 4)
7227                    .unwrap(),
7228            ) as ArrayRef;
7229            push_like(
7230                schema.as_ref(),
7231                "dec_fix_s20_4",
7232                dec20_4,
7233                &mut fields,
7234                &mut columns,
7235            );
7236        }
7237        {
7238            let it = [Some(uuid1), Some(uuid2), Some(uuid1), Some(uuid2)].into_iter();
7239            let arr =
7240                Arc::new(FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap())
7241                    as ArrayRef;
7242            push_like(schema.as_ref(), "uuid_str", arr, &mut fields, &mut columns);
7243        }
7244        push_like(
7245            schema.as_ref(),
7246            "d_date",
7247            Arc::new(Date32Array::from(vec![date_a, 0, 1, 365])) as ArrayRef,
7248            &mut fields,
7249            &mut columns,
7250        );
7251        push_like(
7252            schema.as_ref(),
7253            "t_millis",
7254            Arc::new(Time32MillisecondArray::from(vec![
7255                time_ms_a,
7256                0,
7257                1,
7258                86_400_000 - 1,
7259            ])) as ArrayRef,
7260            &mut fields,
7261            &mut columns,
7262        );
7263        push_like(
7264            schema.as_ref(),
7265            "t_micros",
7266            Arc::new(Time64MicrosecondArray::from(vec![
7267                time_us_eod,
7268                0,
7269                1,
7270                1_000_000,
7271            ])) as ArrayRef,
7272            &mut fields,
7273            &mut columns,
7274        );
7275        {
7276            let a = TimestampMillisecondArray::from(vec![
7277                ts_ms_2024_01_01,
7278                -1,
7279                ts_ms_2024_01_01 + 123,
7280                0,
7281            ])
7282            .with_timezone("+00:00");
7283            push_like(
7284                schema.as_ref(),
7285                "ts_millis_utc",
7286                Arc::new(a) as ArrayRef,
7287                &mut fields,
7288                &mut columns,
7289            );
7290        }
7291        {
7292            let a = TimestampMicrosecondArray::from(vec![
7293                ts_us_2024_01_01,
7294                1,
7295                ts_us_2024_01_01 + 456,
7296                0,
7297            ])
7298            .with_timezone("+00:00");
7299            push_like(
7300                schema.as_ref(),
7301                "ts_micros_utc",
7302                Arc::new(a) as ArrayRef,
7303                &mut fields,
7304                &mut columns,
7305            );
7306        }
7307        push_like(
7308            schema.as_ref(),
7309            "ts_millis_local",
7310            Arc::new(TimestampMillisecondArray::from(vec![
7311                ts_ms_2024_01_01 + 86_400_000,
7312                0,
7313                ts_ms_2024_01_01 + 789,
7314                123_456_789,
7315            ])) as ArrayRef,
7316            &mut fields,
7317            &mut columns,
7318        );
7319        push_like(
7320            schema.as_ref(),
7321            "ts_micros_local",
7322            Arc::new(TimestampMicrosecondArray::from(vec![
7323                ts_us_2024_01_01 + 123_456,
7324                0,
7325                ts_us_2024_01_01 + 101_112,
7326                987_654_321,
7327            ])) as ArrayRef,
7328            &mut fields,
7329            &mut columns,
7330        );
7331        {
7332            let v = vec![dur_small, dur_zero, dur_large, dur_2years];
7333            push_like(
7334                schema.as_ref(),
7335                "interval_mdn",
7336                Arc::new(IntervalMonthDayNanoArray::from(v)) as ArrayRef,
7337                &mut fields,
7338                &mut columns,
7339            );
7340        }
7341        {
7342            let keys = Int32Array::from(vec![1, 2, 3, 0]); // NEW, PROCESSING, DONE, UNKNOWN
7343            let values = Arc::new(StringArray::from(vec![
7344                "UNKNOWN",
7345                "NEW",
7346                "PROCESSING",
7347                "DONE",
7348            ])) as ArrayRef;
7349            let dict = DictionaryArray::<Int32Type>::try_new(keys, values).unwrap();
7350            push_like(
7351                schema.as_ref(),
7352                "status",
7353                Arc::new(dict) as ArrayRef,
7354                &mut fields,
7355                &mut columns,
7356            );
7357        }
7358        {
7359            let list_field = match schema.field_with_name("arr_union").unwrap().data_type() {
7360                DataType::List(f) => f.clone(),
7361                other => panic!("arr_union should be List, got {other:?}"),
7362            };
7363            let uf = match list_field.data_type() {
7364                DataType::Union(f, UnionMode::Dense) => f.clone(),
7365                other => panic!("arr_union item should be union, got {other:?}"),
7366            };
7367            let tid_l = tid_by_name(&uf, "long");
7368            let tid_s = tid_by_name(&uf, "string");
7369            let tid_n = tid_by_name(&uf, "null");
7370            let type_ids = vec![
7371                tid_l, tid_s, tid_n, tid_l, tid_n, tid_s, tid_l, tid_l, tid_s, tid_n, tid_l,
7372            ];
7373            let offsets = vec![0, 0, 0, 1, 1, 1, 2, 3, 2, 2, 4];
7374            let values = mk_dense_union(&uf, type_ids, offsets, |f| match f.data_type() {
7375                DataType::Int64 => {
7376                    Some(Arc::new(Int64Array::from(vec![1i64, -3, 0, -1, 0])) as ArrayRef)
7377                }
7378                DataType::Utf8 => {
7379                    Some(Arc::new(StringArray::from(vec!["x", "z", "end"])) as ArrayRef)
7380                }
7381                DataType::Null => Some(Arc::new(NullArray::new(3)) as ArrayRef),
7382                _ => None,
7383            });
7384            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 4, 7, 8, 11]));
7385            let arr = Arc::new(ListArray::try_new(list_field, list_offsets, values, None).unwrap())
7386                as ArrayRef;
7387            push_like(schema.as_ref(), "arr_union", arr, &mut fields, &mut columns);
7388        }
7389        {
7390            let (entry_field, entries_fields, uf, is_sorted) =
7391                match schema.field_with_name("map_union").unwrap().data_type() {
7392                    DataType::Map(entry_field, is_sorted) => {
7393                        let fs = match entry_field.data_type() {
7394                            DataType::Struct(fs) => fs.clone(),
7395                            other => panic!("map entries must be struct, got {other:?}"),
7396                        };
7397                        let val_f = fs[1].clone();
7398                        let uf = match val_f.data_type() {
7399                            DataType::Union(f, UnionMode::Dense) => f.clone(),
7400                            other => panic!("map value must be union, got {other:?}"),
7401                        };
7402                        (entry_field.clone(), fs, uf, *is_sorted)
7403                    }
7404                    other => panic!("map_union should be Map, got {other:?}"),
7405                };
7406            let keys = StringArray::from(vec!["a", "b", "c", "neg", "pi", "ok"]);
7407            let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4, 4, 6]));
7408            let tid_null = tid_by_name(&uf, "null");
7409            let tid_d = tid_by_name(&uf, "double");
7410            let tid_s = tid_by_name(&uf, "string");
7411            let type_ids = vec![tid_d, tid_null, tid_s, tid_d, tid_d, tid_s];
7412            let offsets = vec![0, 0, 0, 1, 2, 1];
7413            let pi_5dp = (std::f64::consts::PI * 100_000.0).trunc() / 100_000.0;
7414            let vals = mk_dense_union(&uf, type_ids, offsets, |f| match f.data_type() {
7415                DataType::Float64 => {
7416                    Some(Arc::new(Float64Array::from(vec![1.5f64, -0.5, pi_5dp])) as ArrayRef)
7417                }
7418                DataType::Utf8 => {
7419                    Some(Arc::new(StringArray::from(vec!["yes", "true"])) as ArrayRef)
7420                }
7421                DataType::Null => Some(Arc::new(NullArray::new(2)) as ArrayRef),
7422                _ => None,
7423            });
7424            let entries = StructArray::new(
7425                entries_fields.clone(),
7426                vec![Arc::new(keys) as ArrayRef, vals],
7427                None,
7428            );
7429            let map =
7430                Arc::new(MapArray::new(entry_field, moff, entries, None, is_sorted)) as ArrayRef;
7431            push_like(schema.as_ref(), "map_union", map, &mut fields, &mut columns);
7432        }
7433        {
7434            let fs = match schema.field_with_name("address").unwrap().data_type() {
7435                DataType::Struct(fs) => fs.clone(),
7436                other => panic!("address should be Struct, got {other:?}"),
7437            };
7438            let street = Arc::new(StringArray::from(vec![
7439                "100 Main",
7440                "",
7441                "42 Galaxy Way",
7442                "End Ave",
7443            ])) as ArrayRef;
7444            let zip = Arc::new(Int32Array::from(vec![12345, 0, 42424, 1])) as ArrayRef;
7445            let country = Arc::new(StringArray::from(vec!["US", "CA", "US", "GB"])) as ArrayRef;
7446            let arr = Arc::new(StructArray::new(fs, vec![street, zip, country], None)) as ArrayRef;
7447            push_like(schema.as_ref(), "address", arr, &mut fields, &mut columns);
7448        }
7449        {
7450            let fs = match schema.field_with_name("maybe_auth").unwrap().data_type() {
7451                DataType::Struct(fs) => fs.clone(),
7452                other => panic!("maybe_auth should be Struct, got {other:?}"),
7453            };
7454            let user =
7455                Arc::new(StringArray::from(vec!["alice", "bob", "carol", "dave"])) as ArrayRef;
7456            let token_values: Vec<Option<&[u8]>> = vec![
7457                None,                           // row 1: null
7458                Some(b"\x01\x02\x03".as_ref()), // row 2: bytes
7459                None,                           // row 3: null
7460                Some(b"".as_ref()),             // row 4: empty bytes
7461            ];
7462            let token = Arc::new(BinaryArray::from(token_values)) as ArrayRef;
7463            let arr = Arc::new(StructArray::new(fs, vec![user, token], None)) as ArrayRef;
7464            push_like(
7465                schema.as_ref(),
7466                "maybe_auth",
7467                arr,
7468                &mut fields,
7469                &mut columns,
7470            );
7471        }
7472        {
7473            let uf = match schema
7474                .field_with_name("union_enum_record_array_map")
7475                .unwrap()
7476                .data_type()
7477            {
7478                DataType::Union(f, UnionMode::Dense) => f.clone(),
7479                other => panic!("union_enum_record_array_map should be union, got {other:?}"),
7480            };
7481            let mut tid_enum: Option<i8> = None;
7482            let mut tid_rec_a: Option<i8> = None;
7483            let mut tid_array: Option<i8> = None;
7484            let mut tid_map: Option<i8> = None;
7485            let mut map_entry_field: Option<FieldRef> = None;
7486            let mut map_sorted: bool = false;
7487            for (tid, f) in uf.iter() {
7488                match f.data_type() {
7489                    DataType::Dictionary(_, _) => tid_enum = Some(tid),
7490                    DataType::Struct(childs)
7491                        if childs.len() == 2
7492                            && childs[0].name() == "a"
7493                            && childs[1].name() == "b" =>
7494                    {
7495                        tid_rec_a = Some(tid)
7496                    }
7497                    DataType::List(item) if matches!(item.data_type(), DataType::Int64) => {
7498                        tid_array = Some(tid)
7499                    }
7500                    DataType::Map(ef, is_sorted) => {
7501                        tid_map = Some(tid);
7502                        map_entry_field = Some(ef.clone());
7503                        map_sorted = *is_sorted;
7504                    }
7505                    _ => {}
7506                }
7507            }
7508            let (tid_enum, tid_rec_a, tid_array, tid_map) = (
7509                tid_enum.unwrap(),
7510                tid_rec_a.unwrap(),
7511                tid_array.unwrap(),
7512                tid_map.unwrap(),
7513            );
7514            let tids = vec![tid_enum, tid_rec_a, tid_array, tid_map];
7515            let offs = vec![0, 0, 0, 0];
7516            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7517                DataType::Dictionary(_, _) => {
7518                    let keys = Int32Array::from(vec![0i32]);
7519                    let values =
7520                        Arc::new(StringArray::from(vec!["RED", "GREEN", "BLUE"])) as ArrayRef;
7521                    Some(
7522                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
7523                            as ArrayRef,
7524                    )
7525                }
7526                DataType::Struct(fs)
7527                    if fs.len() == 2 && fs[0].name() == "a" && fs[1].name() == "b" =>
7528                {
7529                    let a = Int32Array::from(vec![7]);
7530                    let b = StringArray::from(vec!["rec"]);
7531                    Some(Arc::new(StructArray::new(
7532                        fs.clone(),
7533                        vec![Arc::new(a), Arc::new(b)],
7534                        None,
7535                    )) as ArrayRef)
7536                }
7537                DataType::List(field) => {
7538                    let values = Int64Array::from(vec![1i64, 2, 3]);
7539                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3]));
7540                    Some(Arc::new(
7541                        ListArray::try_new(field.clone(), offsets, Arc::new(values), None).unwrap(),
7542                    ) as ArrayRef)
7543                }
7544                DataType::Map(_, _) => {
7545                    let entry_field = map_entry_field.clone().unwrap();
7546                    let (key_field, val_field) = match entry_field.data_type() {
7547                        DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
7548                        _ => unreachable!(),
7549                    };
7550                    let keys = StringArray::from(vec!["k"]);
7551                    let vals = StringArray::from(vec!["v"]);
7552                    let entries = StructArray::new(
7553                        Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
7554                        vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
7555                        None,
7556                    );
7557                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 1]));
7558                    Some(Arc::new(MapArray::new(
7559                        entry_field.clone(),
7560                        offsets,
7561                        entries,
7562                        None,
7563                        map_sorted,
7564                    )) as ArrayRef)
7565                }
7566                _ => None,
7567            });
7568            push_like(
7569                schema.as_ref(),
7570                "union_enum_record_array_map",
7571                arr,
7572                &mut fields,
7573                &mut columns,
7574            );
7575        }
7576        {
7577            let uf = match schema
7578                .field_with_name("union_date_or_fixed4")
7579                .unwrap()
7580                .data_type()
7581            {
7582                DataType::Union(f, UnionMode::Dense) => f.clone(),
7583                other => panic!("union_date_or_fixed4 should be union, got {other:?}"),
7584            };
7585            let tid_date = tid_by_dt(&uf, |dt| matches!(dt, DataType::Date32));
7586            let tid_fx4 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(4)));
7587            let tids = vec![tid_date, tid_fx4, tid_date, tid_fx4];
7588            let offs = vec![0, 0, 1, 1];
7589            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7590                DataType::Date32 => Some(Arc::new(Date32Array::from(vec![date_a, 0])) as ArrayRef),
7591                DataType::FixedSizeBinary(4) => {
7592                    let it = [Some(*b"\x00\x11\x22\x33"), Some(*b"ABCD")].into_iter();
7593                    Some(Arc::new(
7594                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 4).unwrap(),
7595                    ) as ArrayRef)
7596                }
7597                _ => None,
7598            });
7599            push_like(
7600                schema.as_ref(),
7601                "union_date_or_fixed4",
7602                arr,
7603                &mut fields,
7604                &mut columns,
7605            );
7606        }
7607        {
7608            let uf = match schema
7609                .field_with_name("union_interval_or_string")
7610                .unwrap()
7611                .data_type()
7612            {
7613                DataType::Union(f, UnionMode::Dense) => f.clone(),
7614                other => panic!("union_interval_or_string should be union, got {other:?}"),
7615            };
7616            let tid_dur = tid_by_dt(&uf, |dt| {
7617                matches!(dt, DataType::Interval(IntervalUnit::MonthDayNano))
7618            });
7619            let tid_str = tid_by_dt(&uf, |dt| matches!(dt, DataType::Utf8));
7620            let tids = vec![tid_dur, tid_str, tid_dur, tid_str];
7621            let offs = vec![0, 0, 1, 1];
7622            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7623                DataType::Interval(IntervalUnit::MonthDayNano) => Some(Arc::new(
7624                    IntervalMonthDayNanoArray::from(vec![dur_small, dur_large]),
7625                )
7626                    as ArrayRef),
7627                DataType::Utf8 => Some(Arc::new(StringArray::from(vec![
7628                    "duration-as-text",
7629                    "iso-8601-period-P1Y",
7630                ])) as ArrayRef),
7631                _ => None,
7632            });
7633            push_like(
7634                schema.as_ref(),
7635                "union_interval_or_string",
7636                arr,
7637                &mut fields,
7638                &mut columns,
7639            );
7640        }
7641        {
7642            let uf = match schema
7643                .field_with_name("union_uuid_or_fixed10")
7644                .unwrap()
7645                .data_type()
7646            {
7647                DataType::Union(f, UnionMode::Dense) => f.clone(),
7648                other => panic!("union_uuid_or_fixed10 should be union, got {other:?}"),
7649            };
7650            let tid_uuid = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(16)));
7651            let tid_fx10 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(10)));
7652            let tids = vec![tid_uuid, tid_fx10, tid_uuid, tid_fx10];
7653            let offs = vec![0, 0, 1, 1];
7654            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7655                DataType::FixedSizeBinary(16) => {
7656                    let it = [Some(uuid1), Some(uuid2)].into_iter();
7657                    Some(Arc::new(
7658                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
7659                    ) as ArrayRef)
7660                }
7661                DataType::FixedSizeBinary(10) => {
7662                    let fx10_a = [0xAAu8; 10];
7663                    let fx10_b = [0x00u8, 0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88, 0x99];
7664                    let it = [Some(fx10_a), Some(fx10_b)].into_iter();
7665                    Some(Arc::new(
7666                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 10).unwrap(),
7667                    ) as ArrayRef)
7668                }
7669                _ => None,
7670            });
7671            push_like(
7672                schema.as_ref(),
7673                "union_uuid_or_fixed10",
7674                arr,
7675                &mut fields,
7676                &mut columns,
7677            );
7678        }
7679        {
7680            let list_field = match schema
7681                .field_with_name("array_records_with_union")
7682                .unwrap()
7683                .data_type()
7684            {
7685                DataType::List(f) => f.clone(),
7686                other => panic!("array_records_with_union should be List, got {other:?}"),
7687            };
7688            let kv_fields = match list_field.data_type() {
7689                DataType::Struct(fs) => fs.clone(),
7690                other => panic!("array_records_with_union items must be Struct, got {other:?}"),
7691            };
7692            let val_field = kv_fields
7693                .iter()
7694                .find(|f| f.name() == "val")
7695                .unwrap()
7696                .clone();
7697            let uf = match val_field.data_type() {
7698                DataType::Union(f, UnionMode::Dense) => f.clone(),
7699                other => panic!("KV.val should be union, got {other:?}"),
7700            };
7701            let keys = Arc::new(StringArray::from(vec!["k1", "k2", "k", "k3", "x"])) as ArrayRef;
7702            let tid_null = tid_by_name(&uf, "null");
7703            let tid_i = tid_by_name(&uf, "int");
7704            let tid_l = tid_by_name(&uf, "long");
7705            let type_ids = vec![tid_i, tid_null, tid_l, tid_null, tid_i];
7706            let offsets = vec![0, 0, 0, 1, 1];
7707            let vals = mk_dense_union(&uf, type_ids, offsets, |f| match f.data_type() {
7708                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![5, -5])) as ArrayRef),
7709                DataType::Int64 => Some(Arc::new(Int64Array::from(vec![99i64])) as ArrayRef),
7710                DataType::Null => Some(Arc::new(NullArray::new(2)) as ArrayRef),
7711                _ => None,
7712            });
7713            let values_struct =
7714                Arc::new(StructArray::new(kv_fields.clone(), vec![keys, vals], None)) as ArrayRef;
7715            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3, 4, 5]));
7716            let arr = Arc::new(
7717                ListArray::try_new(list_field, list_offsets, values_struct, None).unwrap(),
7718            ) as ArrayRef;
7719            push_like(
7720                schema.as_ref(),
7721                "array_records_with_union",
7722                arr,
7723                &mut fields,
7724                &mut columns,
7725            );
7726        }
7727        {
7728            let uf = match schema
7729                .field_with_name("union_map_or_array_int")
7730                .unwrap()
7731                .data_type()
7732            {
7733                DataType::Union(f, UnionMode::Dense) => f.clone(),
7734                other => panic!("union_map_or_array_int should be union, got {other:?}"),
7735            };
7736            let tid_map = tid_by_dt(&uf, |dt| matches!(dt, DataType::Map(_, _)));
7737            let tid_list = tid_by_dt(&uf, |dt| matches!(dt, DataType::List(_)));
7738            let map_child: ArrayRef = {
7739                let (entry_field, is_sorted) = match uf
7740                    .iter()
7741                    .find(|(tid, _)| *tid == tid_map)
7742                    .unwrap()
7743                    .1
7744                    .data_type()
7745                {
7746                    DataType::Map(ef, is_sorted) => (ef.clone(), *is_sorted),
7747                    _ => unreachable!(),
7748                };
7749                let (key_field, val_field) = match entry_field.data_type() {
7750                    DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
7751                    _ => unreachable!(),
7752                };
7753                let keys = StringArray::from(vec!["x", "y", "only"]);
7754                let vals = Int32Array::from(vec![1, 2, 10]);
7755                let entries = StructArray::new(
7756                    Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
7757                    vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
7758                    None,
7759                );
7760                let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3]));
7761                Arc::new(MapArray::new(entry_field, moff, entries, None, is_sorted)) as ArrayRef
7762            };
7763            let list_child: ArrayRef = {
7764                let list_field = match uf
7765                    .iter()
7766                    .find(|(tid, _)| *tid == tid_list)
7767                    .unwrap()
7768                    .1
7769                    .data_type()
7770                {
7771                    DataType::List(f) => f.clone(),
7772                    _ => unreachable!(),
7773                };
7774                let values = Int32Array::from(vec![1, 2, 3, 0]);
7775                let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4]));
7776                Arc::new(ListArray::try_new(list_field, offsets, Arc::new(values), None).unwrap())
7777                    as ArrayRef
7778            };
7779            let tids = vec![tid_map, tid_list, tid_map, tid_list];
7780            let offs = vec![0, 0, 1, 1];
7781            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7782                DataType::Map(_, _) => Some(map_child.clone()),
7783                DataType::List(_) => Some(list_child.clone()),
7784                _ => None,
7785            });
7786            push_like(
7787                schema.as_ref(),
7788                "union_map_or_array_int",
7789                arr,
7790                &mut fields,
7791                &mut columns,
7792            );
7793        }
7794        push_like(
7795            schema.as_ref(),
7796            "renamed_with_default",
7797            Arc::new(Int32Array::from(vec![100, 42, 7, 42])) as ArrayRef,
7798            &mut fields,
7799            &mut columns,
7800        );
7801        {
7802            let fs = match schema.field_with_name("person").unwrap().data_type() {
7803                DataType::Struct(fs) => fs.clone(),
7804                other => panic!("person should be Struct, got {other:?}"),
7805            };
7806            let name =
7807                Arc::new(StringArray::from(vec!["Alice", "Bob", "Carol", "Dave"])) as ArrayRef;
7808            let age = Arc::new(Int32Array::from(vec![30, 0, 25, 41])) as ArrayRef;
7809            let arr = Arc::new(StructArray::new(fs, vec![name, age], None)) as ArrayRef;
7810            push_like(schema.as_ref(), "person", arr, &mut fields, &mut columns);
7811        }
7812        let expected =
7813            RecordBatch::try_new(Arc::new(Schema::new(Fields::from(fields))), columns).unwrap();
7814        assert_eq!(
7815            expected, batch,
7816            "entire RecordBatch mismatch (schema, all columns, all rows)"
7817        );
7818    }
7819    #[test]
7820    fn comprehensive_e2e_resolution_test() {
7821        use serde_json::Value;
7822        use std::collections::HashMap;
7823
7824        // Build a reader schema that stresses Avro schema‑resolution
7825        //
7826        // Changes relative to writer schema:
7827        // * Rename fields using writer aliases:    id -> identifier, renamed_with_default -> old_count
7828        // * Promote numeric types:                 count_i32 (int) -> long, ratio_f32 (float) -> double
7829        // * Reorder many union branches (reverse), incl. nested unions
7830        // * Reorder array/map union item/value branches
7831        // * Rename nested Address field:           street -> street_name (uses alias in writer)
7832        // * Change Person type name/namespace:     com.example.Person (matches writer alias)
7833        // * Reverse top‑level field order
7834        //
7835        // Reader‑side aliases are added wherever names change (per Avro spec).
7836        fn make_comprehensive_reader_schema(path: &str) -> AvroSchema {
7837            fn set_type_string(f: &mut Value, new_ty: &str) {
7838                if let Some(ty) = f.get_mut("type") {
7839                    match ty {
7840                        Value::String(_) | Value::Object(_) => {
7841                            *ty = Value::String(new_ty.to_string());
7842                        }
7843                        Value::Array(arr) => {
7844                            for b in arr.iter_mut() {
7845                                match b {
7846                                    Value::String(s) if s != "null" => {
7847                                        *b = Value::String(new_ty.to_string());
7848                                        break;
7849                                    }
7850                                    Value::Object(_) => {
7851                                        *b = Value::String(new_ty.to_string());
7852                                        break;
7853                                    }
7854                                    _ => {}
7855                                }
7856                            }
7857                        }
7858                        _ => {}
7859                    }
7860                }
7861            }
7862            fn reverse_union_array(f: &mut Value) {
7863                if let Some(arr) = f.get_mut("type").and_then(|t| t.as_array_mut()) {
7864                    arr.reverse();
7865                }
7866            }
7867            fn reverse_items_union(f: &mut Value) {
7868                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
7869                    if let Some(items) = obj.get_mut("items").and_then(|v| v.as_array_mut()) {
7870                        items.reverse();
7871                    }
7872                }
7873            }
7874            fn reverse_map_values_union(f: &mut Value) {
7875                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
7876                    if let Some(values) = obj.get_mut("values").and_then(|v| v.as_array_mut()) {
7877                        values.reverse();
7878                    }
7879                }
7880            }
7881            fn reverse_nested_union_in_record(f: &mut Value, field_name: &str) {
7882                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
7883                    if let Some(fields) = obj.get_mut("fields").and_then(|v| v.as_array_mut()) {
7884                        for ff in fields.iter_mut() {
7885                            if ff.get("name").and_then(|n| n.as_str()) == Some(field_name) {
7886                                if let Some(ty) = ff.get_mut("type") {
7887                                    if let Some(arr) = ty.as_array_mut() {
7888                                        arr.reverse();
7889                                    }
7890                                }
7891                            }
7892                        }
7893                    }
7894                }
7895            }
7896            fn rename_nested_field_with_alias(f: &mut Value, old: &str, new: &str) {
7897                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
7898                    if let Some(fields) = obj.get_mut("fields").and_then(|v| v.as_array_mut()) {
7899                        for ff in fields.iter_mut() {
7900                            if ff.get("name").and_then(|n| n.as_str()) == Some(old) {
7901                                ff["name"] = Value::String(new.to_string());
7902                                ff["aliases"] = Value::Array(vec![Value::String(old.to_string())]);
7903                            }
7904                        }
7905                    }
7906                }
7907            }
7908            let mut root = load_writer_schema_json(path);
7909            assert_eq!(root["type"], "record", "writer schema must be a record");
7910            let fields = root
7911                .get_mut("fields")
7912                .and_then(|f| f.as_array_mut())
7913                .expect("record has fields");
7914            for f in fields.iter_mut() {
7915                let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
7916                    continue;
7917                };
7918                match name {
7919                    // Field aliasing (reader‑side aliases added)
7920                    "id" => {
7921                        f["name"] = Value::String("identifier".into());
7922                        f["aliases"] = Value::Array(vec![Value::String("id".into())]);
7923                    }
7924                    "renamed_with_default" => {
7925                        f["name"] = Value::String("old_count".into());
7926                        f["aliases"] =
7927                            Value::Array(vec![Value::String("renamed_with_default".into())]);
7928                    }
7929                    // Promotions
7930                    "count_i32" => set_type_string(f, "long"),
7931                    "ratio_f32" => set_type_string(f, "double"),
7932                    // Union reorder (exercise resolution)
7933                    "opt_str_nullsecond" => reverse_union_array(f),
7934                    "union_enum_record_array_map" => reverse_union_array(f),
7935                    "union_date_or_fixed4" => reverse_union_array(f),
7936                    "union_interval_or_string" => reverse_union_array(f),
7937                    "union_uuid_or_fixed10" => reverse_union_array(f),
7938                    "union_map_or_array_int" => reverse_union_array(f),
7939                    "maybe_auth" => reverse_nested_union_in_record(f, "token"),
7940                    // Array/Map unions
7941                    "arr_union" => reverse_items_union(f),
7942                    "map_union" => reverse_map_values_union(f),
7943                    // Nested rename using reader‑side alias
7944                    "address" => rename_nested_field_with_alias(f, "street", "street_name"),
7945                    // Type‑name alias for nested record
7946                    "person" => {
7947                        if let Some(tobj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
7948                            tobj.insert("name".to_string(), Value::String("Person".into()));
7949                            tobj.insert(
7950                                "namespace".to_string(),
7951                                Value::String("com.example".into()),
7952                            );
7953                            tobj.insert(
7954                                "aliases".into(),
7955                                Value::Array(vec![
7956                                    Value::String("PersonV2".into()),
7957                                    Value::String("com.example.v2.PersonV2".into()),
7958                                ]),
7959                            );
7960                        }
7961                    }
7962                    _ => {}
7963                }
7964            }
7965            fields.reverse();
7966            AvroSchema::new(root.to_string())
7967        }
7968
7969        let path = "test/data/comprehensive_e2e.avro";
7970        let reader_schema = make_comprehensive_reader_schema(path);
7971        let batch = read_alltypes_with_reader_schema(path, reader_schema.clone());
7972
7973        const UUID_EXT_KEY: &str = "ARROW:extension:name";
7974        const UUID_LOGICAL_KEY: &str = "logicalType";
7975
7976        let uuid_md_top: Option<HashMap<String, String>> = batch
7977            .schema()
7978            .field_with_name("uuid_str")
7979            .ok()
7980            .and_then(|f| {
7981                let md = f.metadata();
7982                let has_ext = md.get(UUID_EXT_KEY).is_some();
7983                let is_uuid_logical = md
7984                    .get(UUID_LOGICAL_KEY)
7985                    .map(|v| v.trim_matches('"') == "uuid")
7986                    .unwrap_or(false);
7987                if has_ext || is_uuid_logical {
7988                    Some(md.clone())
7989                } else {
7990                    None
7991                }
7992            });
7993
7994        let uuid_md_union: Option<HashMap<String, String>> = batch
7995            .schema()
7996            .field_with_name("union_uuid_or_fixed10")
7997            .ok()
7998            .and_then(|f| match f.data_type() {
7999                DataType::Union(uf, _) => uf
8000                    .iter()
8001                    .find(|(_, child)| child.name() == "uuid")
8002                    .and_then(|(_, child)| {
8003                        let md = child.metadata();
8004                        let has_ext = md.get(UUID_EXT_KEY).is_some();
8005                        let is_uuid_logical = md
8006                            .get(UUID_LOGICAL_KEY)
8007                            .map(|v| v.trim_matches('"') == "uuid")
8008                            .unwrap_or(false);
8009                        if has_ext || is_uuid_logical {
8010                            Some(md.clone())
8011                        } else {
8012                            None
8013                        }
8014                    }),
8015                _ => None,
8016            });
8017
8018        let add_uuid_ext_top = |f: Field| -> Field {
8019            if let Some(md) = &uuid_md_top {
8020                f.with_metadata(md.clone())
8021            } else {
8022                f
8023            }
8024        };
8025        let add_uuid_ext_union = |f: Field| -> Field {
8026            if let Some(md) = &uuid_md_union {
8027                f.with_metadata(md.clone())
8028            } else {
8029                f
8030            }
8031        };
8032
8033        #[inline]
8034        fn uuid16_from_str(s: &str) -> [u8; 16] {
8035            let mut out = [0u8; 16];
8036            let mut idx = 0usize;
8037            let mut hi: Option<u8> = None;
8038            for ch in s.chars() {
8039                if ch == '-' {
8040                    continue;
8041                }
8042                let v = ch.to_digit(16).expect("invalid hex digit in UUID") as u8;
8043                if let Some(h) = hi {
8044                    out[idx] = (h << 4) | v;
8045                    idx += 1;
8046                    hi = None;
8047                } else {
8048                    hi = Some(v);
8049                }
8050            }
8051            assert_eq!(idx, 16, "UUID must decode to 16 bytes");
8052            out
8053        }
8054
8055        fn mk_dense_union(
8056            fields: &UnionFields,
8057            type_ids: Vec<i8>,
8058            offsets: Vec<i32>,
8059            provide: impl Fn(&Field) -> Option<ArrayRef>,
8060        ) -> ArrayRef {
8061            fn empty_child_for(dt: &DataType) -> Arc<dyn Array> {
8062                match dt {
8063                    DataType::Null => Arc::new(NullArray::new(0)),
8064                    DataType::Boolean => Arc::new(BooleanArray::from(Vec::<bool>::new())),
8065                    DataType::Int32 => Arc::new(Int32Array::from(Vec::<i32>::new())),
8066                    DataType::Int64 => Arc::new(Int64Array::from(Vec::<i64>::new())),
8067                    DataType::Float32 => Arc::new(Float32Array::from(Vec::<f32>::new())),
8068                    DataType::Float64 => Arc::new(Float64Array::from(Vec::<f64>::new())),
8069                    DataType::Binary => Arc::new(BinaryArray::from(Vec::<&[u8]>::new())),
8070                    DataType::Utf8 => Arc::new(StringArray::from(Vec::<&str>::new())),
8071                    DataType::Date32 => Arc::new(Date32Array::from(Vec::<i32>::new())),
8072                    DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
8073                        Arc::new(Time32MillisecondArray::from(Vec::<i32>::new()))
8074                    }
8075                    DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
8076                        Arc::new(Time64MicrosecondArray::from(Vec::<i64>::new()))
8077                    }
8078                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
8079                        let a = TimestampMillisecondArray::from(Vec::<i64>::new());
8080                        Arc::new(if let Some(tz) = tz {
8081                            a.with_timezone(tz.clone())
8082                        } else {
8083                            a
8084                        })
8085                    }
8086                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
8087                        let a = TimestampMicrosecondArray::from(Vec::<i64>::new());
8088                        Arc::new(if let Some(tz) = tz {
8089                            a.with_timezone(tz.clone())
8090                        } else {
8091                            a
8092                        })
8093                    }
8094                    DataType::Interval(IntervalUnit::MonthDayNano) => Arc::new(
8095                        IntervalMonthDayNanoArray::from(Vec::<IntervalMonthDayNano>::new()),
8096                    ),
8097                    DataType::FixedSizeBinary(sz) => Arc::new(
8098                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(
8099                            std::iter::empty::<Option<Vec<u8>>>(),
8100                            *sz,
8101                        )
8102                        .unwrap(),
8103                    ),
8104                    DataType::Dictionary(_, _) => {
8105                        let keys = Int32Array::from(Vec::<i32>::new());
8106                        let values = Arc::new(StringArray::from(Vec::<&str>::new()));
8107                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
8108                    }
8109                    DataType::Struct(fields) => {
8110                        let children: Vec<ArrayRef> = fields
8111                            .iter()
8112                            .map(|f| empty_child_for(f.data_type()) as ArrayRef)
8113                            .collect();
8114                        Arc::new(StructArray::new(fields.clone(), children, None))
8115                    }
8116                    DataType::List(field) => {
8117                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
8118                        Arc::new(
8119                            ListArray::try_new(
8120                                field.clone(),
8121                                offsets,
8122                                empty_child_for(field.data_type()),
8123                                None,
8124                            )
8125                            .unwrap(),
8126                        )
8127                    }
8128                    DataType::Map(entry_field, is_sorted) => {
8129                        let (key_field, val_field) = match entry_field.data_type() {
8130                            DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
8131                            other => panic!("unexpected map entries type: {other:?}"),
8132                        };
8133                        let keys = StringArray::from(Vec::<&str>::new());
8134                        let vals: ArrayRef = match val_field.data_type() {
8135                            DataType::Null => Arc::new(NullArray::new(0)) as ArrayRef,
8136                            DataType::Boolean => {
8137                                Arc::new(BooleanArray::from(Vec::<bool>::new())) as ArrayRef
8138                            }
8139                            DataType::Int32 => {
8140                                Arc::new(Int32Array::from(Vec::<i32>::new())) as ArrayRef
8141                            }
8142                            DataType::Int64 => {
8143                                Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
8144                            }
8145                            DataType::Float32 => {
8146                                Arc::new(Float32Array::from(Vec::<f32>::new())) as ArrayRef
8147                            }
8148                            DataType::Float64 => {
8149                                Arc::new(Float64Array::from(Vec::<f64>::new())) as ArrayRef
8150                            }
8151                            DataType::Utf8 => {
8152                                Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
8153                            }
8154                            DataType::Binary => {
8155                                Arc::new(BinaryArray::from(Vec::<&[u8]>::new())) as ArrayRef
8156                            }
8157                            DataType::Union(uf, _) => {
8158                                let children: Vec<ArrayRef> = uf
8159                                    .iter()
8160                                    .map(|(_, f)| empty_child_for(f.data_type()))
8161                                    .collect();
8162                                Arc::new(
8163                                    UnionArray::try_new(
8164                                        uf.clone(),
8165                                        ScalarBuffer::<i8>::from(Vec::<i8>::new()),
8166                                        Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
8167                                        children,
8168                                    )
8169                                    .unwrap(),
8170                                ) as ArrayRef
8171                            }
8172                            other => panic!("unsupported map value type: {other:?}"),
8173                        };
8174                        let entries = StructArray::new(
8175                            Fields::from(vec![
8176                                key_field.as_ref().clone(),
8177                                val_field.as_ref().clone(),
8178                            ]),
8179                            vec![Arc::new(keys) as ArrayRef, vals],
8180                            None,
8181                        );
8182                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
8183                        Arc::new(MapArray::new(
8184                            entry_field.clone(),
8185                            offsets,
8186                            entries,
8187                            None,
8188                            *is_sorted,
8189                        ))
8190                    }
8191                    other => panic!("empty_child_for: unhandled type {other:?}"),
8192                }
8193            }
8194            let children: Vec<ArrayRef> = fields
8195                .iter()
8196                .map(|(_, f)| provide(f).unwrap_or_else(|| empty_child_for(f.data_type())))
8197                .collect();
8198            Arc::new(
8199                UnionArray::try_new(
8200                    fields.clone(),
8201                    ScalarBuffer::<i8>::from(type_ids),
8202                    Some(ScalarBuffer::<i32>::from(offsets)),
8203                    children,
8204                )
8205                .unwrap(),
8206            ) as ArrayRef
8207        }
8208        let date_a: i32 = 19_000; // 2022-01-08
8209        let time_ms_a: i32 = 12 * 3_600_000 + 34 * 60_000 + 56_000 + 789;
8210        let time_us_eod: i64 = 86_400_000_000 - 1;
8211        let ts_ms_2024_01_01: i64 = 1_704_067_200_000; // 2024-01-01T00:00:00Z
8212        let ts_us_2024_01_01: i64 = ts_ms_2024_01_01 * 1_000;
8213        let dur_small = IntervalMonthDayNanoType::make_value(1, 2, 3_000_000_000);
8214        let dur_zero = IntervalMonthDayNanoType::make_value(0, 0, 0);
8215        let dur_large =
8216            IntervalMonthDayNanoType::make_value(12, 31, ((86_400_000 - 1) as i64) * 1_000_000);
8217        let dur_2years = IntervalMonthDayNanoType::make_value(24, 0, 0);
8218        let uuid1 = uuid16_from_str("fe7bc30b-4ce8-4c5e-b67c-2234a2d38e66");
8219        let uuid2 = uuid16_from_str("0826cc06-d2e3-4599-b4ad-af5fa6905cdb");
8220        let item_name = Field::LIST_FIELD_DEFAULT_NAME;
8221        let uf_tri = UnionFields::try_new(
8222            vec![0, 1, 2],
8223            vec![
8224                Field::new("int", DataType::Int32, false),
8225                Field::new("string", DataType::Utf8, false),
8226                Field::new("boolean", DataType::Boolean, false),
8227            ],
8228        )
8229        .unwrap();
8230        let uf_arr_items = UnionFields::try_new(
8231            vec![0, 1, 2],
8232            vec![
8233                Field::new("null", DataType::Null, false),
8234                Field::new("string", DataType::Utf8, false),
8235                Field::new("long", DataType::Int64, false),
8236            ],
8237        )
8238        .unwrap();
8239        let arr_items_field = Arc::new(Field::new(
8240            item_name,
8241            DataType::Union(uf_arr_items.clone(), UnionMode::Dense),
8242            true,
8243        ));
8244        let uf_map_vals = UnionFields::try_new(
8245            vec![0, 1, 2],
8246            vec![
8247                Field::new("string", DataType::Utf8, false),
8248                Field::new("double", DataType::Float64, false),
8249                Field::new("null", DataType::Null, false),
8250            ],
8251        )
8252        .unwrap();
8253        let map_entries_field = Arc::new(Field::new(
8254            "entries",
8255            DataType::Struct(Fields::from(vec![
8256                Field::new("key", DataType::Utf8, false),
8257                Field::new(
8258                    "value",
8259                    DataType::Union(uf_map_vals.clone(), UnionMode::Dense),
8260                    true,
8261                ),
8262            ])),
8263            false,
8264        ));
8265        // Enum metadata for Color (now includes name/namespace)
8266        let mut enum_md_color = {
8267            let mut m = HashMap::<String, String>::new();
8268            m.insert(
8269                crate::schema::AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
8270                serde_json::to_string(&vec!["RED", "GREEN", "BLUE"]).unwrap(),
8271            );
8272            m
8273        };
8274        enum_md_color.insert(AVRO_NAME_METADATA_KEY.to_string(), "Color".to_string());
8275        enum_md_color.insert(
8276            AVRO_NAMESPACE_METADATA_KEY.to_string(),
8277            "org.apache.arrow.avrotests.v1.types".to_string(),
8278        );
8279        let union_rec_a_fields = Fields::from(vec![
8280            Field::new("a", DataType::Int32, false),
8281            Field::new("b", DataType::Utf8, false),
8282        ]);
8283        let union_rec_b_fields = Fields::from(vec![
8284            Field::new("x", DataType::Int64, false),
8285            Field::new("y", DataType::Binary, false),
8286        ]);
8287        let union_map_entries = Arc::new(Field::new(
8288            "entries",
8289            DataType::Struct(Fields::from(vec![
8290                Field::new("key", DataType::Utf8, false),
8291                Field::new("value", DataType::Utf8, false),
8292            ])),
8293            false,
8294        ));
8295        let rec_a_md = {
8296            let mut m = HashMap::<String, String>::new();
8297            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "RecA".to_string());
8298            m.insert(
8299                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8300                "org.apache.arrow.avrotests.v1.types".to_string(),
8301            );
8302            m
8303        };
8304        let rec_b_md = {
8305            let mut m = HashMap::<String, String>::new();
8306            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "RecB".to_string());
8307            m.insert(
8308                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8309                "org.apache.arrow.avrotests.v1.types".to_string(),
8310            );
8311            m
8312        };
8313        let uf_union_big = UnionFields::try_new(
8314            vec![0, 1, 2, 3, 4],
8315            vec![
8316                Field::new(
8317                    "map",
8318                    DataType::Map(union_map_entries.clone(), false),
8319                    false,
8320                ),
8321                Field::new(
8322                    "array",
8323                    DataType::List(Arc::new(Field::new(item_name, DataType::Int64, false))),
8324                    false,
8325                ),
8326                Field::new(
8327                    "org.apache.arrow.avrotests.v1.types.RecB",
8328                    DataType::Struct(union_rec_b_fields.clone()),
8329                    false,
8330                )
8331                .with_metadata(rec_b_md.clone()),
8332                Field::new(
8333                    "org.apache.arrow.avrotests.v1.types.RecA",
8334                    DataType::Struct(union_rec_a_fields.clone()),
8335                    false,
8336                )
8337                .with_metadata(rec_a_md.clone()),
8338                Field::new(
8339                    "org.apache.arrow.avrotests.v1.types.Color",
8340                    DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
8341                    false,
8342                )
8343                .with_metadata(enum_md_color.clone()),
8344            ],
8345        )
8346        .unwrap();
8347        let fx4_md = {
8348            let mut m = HashMap::<String, String>::new();
8349            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Fx4".to_string());
8350            m.insert(
8351                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8352                "org.apache.arrow.avrotests.v1".to_string(),
8353            );
8354            m
8355        };
8356        let uf_date_fixed4 = UnionFields::try_new(
8357            vec![0, 1],
8358            vec![
8359                Field::new(
8360                    "org.apache.arrow.avrotests.v1.Fx4",
8361                    DataType::FixedSizeBinary(4),
8362                    false,
8363                )
8364                .with_metadata(fx4_md.clone()),
8365                Field::new("date", DataType::Date32, false),
8366            ],
8367        )
8368        .unwrap();
8369        let dur12u_md = {
8370            let mut m = HashMap::<String, String>::new();
8371            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Dur12U".to_string());
8372            m.insert(
8373                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8374                "org.apache.arrow.avrotests.v1".to_string(),
8375            );
8376            m
8377        };
8378        let uf_dur_or_str = UnionFields::try_new(
8379            vec![0, 1],
8380            vec![
8381                Field::new("string", DataType::Utf8, false),
8382                Field::new(
8383                    "org.apache.arrow.avrotests.v1.Dur12U",
8384                    DataType::Interval(arrow_schema::IntervalUnit::MonthDayNano),
8385                    false,
8386                )
8387                .with_metadata(dur12u_md.clone()),
8388            ],
8389        )
8390        .unwrap();
8391        let fx10_md = {
8392            let mut m = HashMap::<String, String>::new();
8393            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Fx10".to_string());
8394            m.insert(
8395                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8396                "org.apache.arrow.avrotests.v1".to_string(),
8397            );
8398            m
8399        };
8400        let uf_uuid_or_fx10 = UnionFields::try_new(
8401            vec![0, 1],
8402            vec![
8403                Field::new(
8404                    "org.apache.arrow.avrotests.v1.Fx10",
8405                    DataType::FixedSizeBinary(10),
8406                    false,
8407                )
8408                .with_metadata(fx10_md.clone()),
8409                add_uuid_ext_union(Field::new("uuid", DataType::FixedSizeBinary(16), false)),
8410            ],
8411        )
8412        .unwrap();
8413        let uf_kv_val = UnionFields::try_new(
8414            vec![0, 1, 2],
8415            vec![
8416                Field::new("null", DataType::Null, false),
8417                Field::new("int", DataType::Int32, false),
8418                Field::new("long", DataType::Int64, false),
8419            ],
8420        )
8421        .unwrap();
8422        let kv_fields = Fields::from(vec![
8423            Field::new("key", DataType::Utf8, false),
8424            Field::new(
8425                "val",
8426                DataType::Union(uf_kv_val.clone(), UnionMode::Dense),
8427                true,
8428            ),
8429        ]);
8430        let kv_item_field = Arc::new(Field::new(
8431            item_name,
8432            DataType::Struct(kv_fields.clone()),
8433            false,
8434        ));
8435        let map_int_entries = Arc::new(Field::new(
8436            "entries",
8437            DataType::Struct(Fields::from(vec![
8438                Field::new("key", DataType::Utf8, false),
8439                Field::new("value", DataType::Int32, false),
8440            ])),
8441            false,
8442        ));
8443        let uf_map_or_array = UnionFields::try_new(
8444            vec![0, 1],
8445            vec![
8446                Field::new(
8447                    "array",
8448                    DataType::List(Arc::new(Field::new(item_name, DataType::Int32, false))),
8449                    false,
8450                ),
8451                Field::new("map", DataType::Map(map_int_entries.clone(), false), false),
8452            ],
8453        )
8454        .unwrap();
8455        let mut enum_md_status = {
8456            let mut m = HashMap::<String, String>::new();
8457            m.insert(
8458                crate::schema::AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
8459                serde_json::to_string(&vec!["UNKNOWN", "NEW", "PROCESSING", "DONE"]).unwrap(),
8460            );
8461            m
8462        };
8463        enum_md_status.insert(AVRO_NAME_METADATA_KEY.to_string(), "Status".to_string());
8464        enum_md_status.insert(
8465            AVRO_NAMESPACE_METADATA_KEY.to_string(),
8466            "org.apache.arrow.avrotests.v1.types".to_string(),
8467        );
8468        let mut dec20_md = HashMap::<String, String>::new();
8469        dec20_md.insert("precision".to_string(), "20".to_string());
8470        dec20_md.insert("scale".to_string(), "4".to_string());
8471        dec20_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "DecFix20".to_string());
8472        dec20_md.insert(
8473            AVRO_NAMESPACE_METADATA_KEY.to_string(),
8474            "org.apache.arrow.avrotests.v1.types".to_string(),
8475        );
8476        let mut dec10_md = HashMap::<String, String>::new();
8477        dec10_md.insert("precision".to_string(), "10".to_string());
8478        dec10_md.insert("scale".to_string(), "2".to_string());
8479        let fx16_top_md = {
8480            let mut m = HashMap::<String, String>::new();
8481            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Fx16".to_string());
8482            m.insert(
8483                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8484                "org.apache.arrow.avrotests.v1.types".to_string(),
8485            );
8486            m
8487        };
8488        let dur12_top_md = {
8489            let mut m = HashMap::<String, String>::new();
8490            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Dur12".to_string());
8491            m.insert(
8492                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8493                "org.apache.arrow.avrotests.v1.types".to_string(),
8494            );
8495            m
8496        };
8497        #[cfg(feature = "small_decimals")]
8498        let dec20_dt = DataType::Decimal128(20, 4);
8499        #[cfg(not(feature = "small_decimals"))]
8500        let dec20_dt = DataType::Decimal128(20, 4);
8501        #[cfg(feature = "small_decimals")]
8502        let dec10_dt = DataType::Decimal64(10, 2);
8503        #[cfg(not(feature = "small_decimals"))]
8504        let dec10_dt = DataType::Decimal128(10, 2);
8505        let fields: Vec<FieldRef> = vec![
8506            Arc::new(Field::new(
8507                "person",
8508                DataType::Struct(Fields::from(vec![
8509                    Field::new("name", DataType::Utf8, false),
8510                    Field::new("age", DataType::Int32, false),
8511                ])),
8512                false,
8513            )),
8514            Arc::new(Field::new("old_count", DataType::Int32, false)),
8515            Arc::new(Field::new(
8516                "union_map_or_array_int",
8517                DataType::Union(uf_map_or_array.clone(), UnionMode::Dense),
8518                false,
8519            )),
8520            Arc::new(Field::new(
8521                "array_records_with_union",
8522                DataType::List(kv_item_field.clone()),
8523                false,
8524            )),
8525            Arc::new(Field::new(
8526                "union_uuid_or_fixed10",
8527                DataType::Union(uf_uuid_or_fx10.clone(), UnionMode::Dense),
8528                false,
8529            )),
8530            Arc::new(Field::new(
8531                "union_interval_or_string",
8532                DataType::Union(uf_dur_or_str.clone(), UnionMode::Dense),
8533                false,
8534            )),
8535            Arc::new(Field::new(
8536                "union_date_or_fixed4",
8537                DataType::Union(uf_date_fixed4.clone(), UnionMode::Dense),
8538                false,
8539            )),
8540            Arc::new(Field::new(
8541                "union_enum_record_array_map",
8542                DataType::Union(uf_union_big.clone(), UnionMode::Dense),
8543                false,
8544            )),
8545            Arc::new(Field::new(
8546                "maybe_auth",
8547                DataType::Struct(Fields::from(vec![
8548                    Field::new("user", DataType::Utf8, false),
8549                    Field::new("token", DataType::Binary, true), // [bytes,null] -> nullable bytes
8550                ])),
8551                false,
8552            )),
8553            Arc::new(Field::new(
8554                "address",
8555                DataType::Struct(Fields::from(vec![
8556                    Field::new("street_name", DataType::Utf8, false),
8557                    Field::new("zip", DataType::Int32, false),
8558                    Field::new("country", DataType::Utf8, false),
8559                ])),
8560                false,
8561            )),
8562            Arc::new(Field::new(
8563                "map_union",
8564                DataType::Map(map_entries_field.clone(), false),
8565                false,
8566            )),
8567            Arc::new(Field::new(
8568                "arr_union",
8569                DataType::List(arr_items_field.clone()),
8570                false,
8571            )),
8572            Arc::new(
8573                Field::new(
8574                    "status",
8575                    DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
8576                    false,
8577                )
8578                .with_metadata(enum_md_status.clone()),
8579            ),
8580            Arc::new(
8581                Field::new(
8582                    "interval_mdn",
8583                    DataType::Interval(IntervalUnit::MonthDayNano),
8584                    false,
8585                )
8586                .with_metadata(dur12_top_md.clone()),
8587            ),
8588            Arc::new(Field::new(
8589                "ts_micros_local",
8590                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None),
8591                false,
8592            )),
8593            Arc::new(Field::new(
8594                "ts_millis_local",
8595                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None),
8596                false,
8597            )),
8598            Arc::new(Field::new(
8599                "ts_micros_utc",
8600                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, Some("+00:00".into())),
8601                false,
8602            )),
8603            Arc::new(Field::new(
8604                "ts_millis_utc",
8605                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, Some("+00:00".into())),
8606                false,
8607            )),
8608            Arc::new(Field::new(
8609                "t_micros",
8610                DataType::Time64(arrow_schema::TimeUnit::Microsecond),
8611                false,
8612            )),
8613            Arc::new(Field::new(
8614                "t_millis",
8615                DataType::Time32(arrow_schema::TimeUnit::Millisecond),
8616                false,
8617            )),
8618            Arc::new(Field::new("d_date", DataType::Date32, false)),
8619            Arc::new(add_uuid_ext_top(Field::new(
8620                "uuid_str",
8621                DataType::FixedSizeBinary(16),
8622                false,
8623            ))),
8624            Arc::new(Field::new("dec_fix_s20_4", dec20_dt, false).with_metadata(dec20_md.clone())),
8625            Arc::new(
8626                Field::new("dec_bytes_s10_2", dec10_dt, false).with_metadata(dec10_md.clone()),
8627            ),
8628            Arc::new(
8629                Field::new("fx16_plain", DataType::FixedSizeBinary(16), false)
8630                    .with_metadata(fx16_top_md.clone()),
8631            ),
8632            Arc::new(Field::new("raw_bytes", DataType::Binary, false)),
8633            Arc::new(Field::new("str_utf8", DataType::Utf8, false)),
8634            Arc::new(Field::new(
8635                "tri_union_prim",
8636                DataType::Union(uf_tri.clone(), UnionMode::Dense),
8637                false,
8638            )),
8639            Arc::new(Field::new("opt_str_nullsecond", DataType::Utf8, true)),
8640            Arc::new(Field::new("opt_i32_nullfirst", DataType::Int32, true)),
8641            Arc::new(Field::new("count_i64", DataType::Int64, false)),
8642            Arc::new(Field::new("count_i32", DataType::Int64, false)),
8643            Arc::new(Field::new("ratio_f64", DataType::Float64, false)),
8644            Arc::new(Field::new("ratio_f32", DataType::Float64, false)),
8645            Arc::new(Field::new("flag", DataType::Boolean, false)),
8646            Arc::new(Field::new("identifier", DataType::Int64, false)),
8647        ];
8648        let expected_schema = Arc::new(arrow_schema::Schema::new(Fields::from(fields)));
8649        let mut cols: Vec<ArrayRef> = vec![
8650            Arc::new(StructArray::new(
8651                match expected_schema
8652                    .field_with_name("person")
8653                    .unwrap()
8654                    .data_type()
8655                {
8656                    DataType::Struct(fs) => fs.clone(),
8657                    _ => unreachable!(),
8658                },
8659                vec![
8660                    Arc::new(StringArray::from(vec!["Alice", "Bob", "Carol", "Dave"])) as ArrayRef,
8661                    Arc::new(Int32Array::from(vec![30, 0, 25, 41])) as ArrayRef,
8662                ],
8663                None,
8664            )) as ArrayRef,
8665            Arc::new(Int32Array::from(vec![100, 42, 7, 42])) as ArrayRef,
8666        ];
8667        {
8668            let map_child: ArrayRef = {
8669                let keys = StringArray::from(vec!["x", "y", "only"]);
8670                let vals = Int32Array::from(vec![1, 2, 10]);
8671                let entries = StructArray::new(
8672                    Fields::from(vec![
8673                        Field::new("key", DataType::Utf8, false),
8674                        Field::new("value", DataType::Int32, false),
8675                    ]),
8676                    vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
8677                    None,
8678                );
8679                let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3]));
8680                Arc::new(MapArray::new(
8681                    map_int_entries.clone(),
8682                    moff,
8683                    entries,
8684                    None,
8685                    false,
8686                )) as ArrayRef
8687            };
8688            let list_child: ArrayRef = {
8689                let values = Int32Array::from(vec![1, 2, 3, 0]);
8690                let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4]));
8691                Arc::new(
8692                    ListArray::try_new(
8693                        Arc::new(Field::new(item_name, DataType::Int32, false)),
8694                        offsets,
8695                        Arc::new(values),
8696                        None,
8697                    )
8698                    .unwrap(),
8699                ) as ArrayRef
8700            };
8701            let tids = vec![1, 0, 1, 0];
8702            let offs = vec![0, 0, 1, 1];
8703            let arr = mk_dense_union(&uf_map_or_array, tids, offs, |f| match f.name().as_str() {
8704                "array" => Some(list_child.clone()),
8705                "map" => Some(map_child.clone()),
8706                _ => None,
8707            });
8708            cols.push(arr);
8709        }
8710        {
8711            let keys = Arc::new(StringArray::from(vec!["k1", "k2", "k", "k3", "x"])) as ArrayRef;
8712            let type_ids = vec![1, 0, 2, 0, 1];
8713            let offsets = vec![0, 0, 0, 1, 1];
8714            let vals = mk_dense_union(&uf_kv_val, type_ids, offsets, |f| match f.data_type() {
8715                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![5, -5])) as ArrayRef),
8716                DataType::Int64 => Some(Arc::new(Int64Array::from(vec![99i64])) as ArrayRef),
8717                DataType::Null => Some(Arc::new(NullArray::new(2)) as ArrayRef),
8718                _ => None,
8719            });
8720            let values_struct =
8721                Arc::new(StructArray::new(kv_fields.clone(), vec![keys, vals], None));
8722            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3, 4, 5]));
8723            let arr = Arc::new(
8724                ListArray::try_new(kv_item_field.clone(), list_offsets, values_struct, None)
8725                    .unwrap(),
8726            ) as ArrayRef;
8727            cols.push(arr);
8728        }
8729        {
8730            let type_ids = vec![1, 0, 1, 0]; // [uuid, fixed10, uuid, fixed10] but uf order = [fixed10, uuid]
8731            let offs = vec![0, 0, 1, 1];
8732            let arr = mk_dense_union(&uf_uuid_or_fx10, type_ids, offs, |f| match f.data_type() {
8733                DataType::FixedSizeBinary(16) => {
8734                    let it = [Some(uuid1), Some(uuid2)].into_iter();
8735                    Some(Arc::new(
8736                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
8737                    ) as ArrayRef)
8738                }
8739                DataType::FixedSizeBinary(10) => {
8740                    let fx10_a = [0xAAu8; 10];
8741                    let fx10_b = [0x00u8, 0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88, 0x99];
8742                    let it = [Some(fx10_a), Some(fx10_b)].into_iter();
8743                    Some(Arc::new(
8744                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 10).unwrap(),
8745                    ) as ArrayRef)
8746                }
8747                _ => None,
8748            });
8749            cols.push(arr);
8750        }
8751        {
8752            let type_ids = vec![1, 0, 1, 0]; // [duration, string, duration, string] but uf order = [string, duration]
8753            let offs = vec![0, 0, 1, 1];
8754            let arr = mk_dense_union(&uf_dur_or_str, type_ids, offs, |f| match f.data_type() {
8755                DataType::Interval(arrow_schema::IntervalUnit::MonthDayNano) => Some(Arc::new(
8756                    IntervalMonthDayNanoArray::from(vec![dur_small, dur_large]),
8757                )
8758                    as ArrayRef),
8759                DataType::Utf8 => Some(Arc::new(StringArray::from(vec![
8760                    "duration-as-text",
8761                    "iso-8601-period-P1Y",
8762                ])) as ArrayRef),
8763                _ => None,
8764            });
8765            cols.push(arr);
8766        }
8767        {
8768            let type_ids = vec![1, 0, 1, 0]; // [date, fixed, date, fixed] but uf order = [fixed, date]
8769            let offs = vec![0, 0, 1, 1];
8770            let arr = mk_dense_union(&uf_date_fixed4, type_ids, offs, |f| match f.data_type() {
8771                DataType::Date32 => Some(Arc::new(Date32Array::from(vec![date_a, 0])) as ArrayRef),
8772                DataType::FixedSizeBinary(4) => {
8773                    let it = [Some(*b"\x00\x11\x22\x33"), Some(*b"ABCD")].into_iter();
8774                    Some(Arc::new(
8775                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 4).unwrap(),
8776                    ) as ArrayRef)
8777                }
8778                _ => None,
8779            });
8780            cols.push(arr);
8781        }
8782        {
8783            let tids = vec![4, 3, 1, 0]; // uf order = [map(0), array(1), RecB(2), RecA(3), enum(4)]
8784            let offs = vec![0, 0, 0, 0];
8785            let arr = mk_dense_union(&uf_union_big, tids, offs, |f| match f.data_type() {
8786                DataType::Dictionary(_, _) => {
8787                    let keys = Int32Array::from(vec![0i32]);
8788                    let values =
8789                        Arc::new(StringArray::from(vec!["RED", "GREEN", "BLUE"])) as ArrayRef;
8790                    Some(
8791                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
8792                            as ArrayRef,
8793                    )
8794                }
8795                DataType::Struct(fs) if fs == &union_rec_a_fields => {
8796                    let a = Int32Array::from(vec![7]);
8797                    let b = StringArray::from(vec!["rec"]);
8798                    Some(Arc::new(StructArray::new(
8799                        fs.clone(),
8800                        vec![Arc::new(a) as ArrayRef, Arc::new(b) as ArrayRef],
8801                        None,
8802                    )) as ArrayRef)
8803                }
8804                DataType::List(_) => {
8805                    let values = Int64Array::from(vec![1i64, 2, 3]);
8806                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3]));
8807                    Some(Arc::new(
8808                        ListArray::try_new(
8809                            Arc::new(Field::new(item_name, DataType::Int64, false)),
8810                            offsets,
8811                            Arc::new(values),
8812                            None,
8813                        )
8814                        .unwrap(),
8815                    ) as ArrayRef)
8816                }
8817                DataType::Map(_, _) => {
8818                    let keys = StringArray::from(vec!["k"]);
8819                    let vals = StringArray::from(vec!["v"]);
8820                    let entries = StructArray::new(
8821                        Fields::from(vec![
8822                            Field::new("key", DataType::Utf8, false),
8823                            Field::new("value", DataType::Utf8, false),
8824                        ]),
8825                        vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
8826                        None,
8827                    );
8828                    let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 1]));
8829                    Some(Arc::new(MapArray::new(
8830                        union_map_entries.clone(),
8831                        moff,
8832                        entries,
8833                        None,
8834                        false,
8835                    )) as ArrayRef)
8836                }
8837                _ => None,
8838            });
8839            cols.push(arr);
8840        }
8841        {
8842            let fs = match expected_schema
8843                .field_with_name("maybe_auth")
8844                .unwrap()
8845                .data_type()
8846            {
8847                DataType::Struct(fs) => fs.clone(),
8848                _ => unreachable!(),
8849            };
8850            let user =
8851                Arc::new(StringArray::from(vec!["alice", "bob", "carol", "dave"])) as ArrayRef;
8852            let token_values: Vec<Option<&[u8]>> = vec![
8853                None,
8854                Some(b"\x01\x02\x03".as_ref()),
8855                None,
8856                Some(b"".as_ref()),
8857            ];
8858            let token = Arc::new(BinaryArray::from(token_values)) as ArrayRef;
8859            cols.push(Arc::new(StructArray::new(fs, vec![user, token], None)) as ArrayRef);
8860        }
8861        {
8862            let fs = match expected_schema
8863                .field_with_name("address")
8864                .unwrap()
8865                .data_type()
8866            {
8867                DataType::Struct(fs) => fs.clone(),
8868                _ => unreachable!(),
8869            };
8870            let street = Arc::new(StringArray::from(vec![
8871                "100 Main",
8872                "",
8873                "42 Galaxy Way",
8874                "End Ave",
8875            ])) as ArrayRef;
8876            let zip = Arc::new(Int32Array::from(vec![12345, 0, 42424, 1])) as ArrayRef;
8877            let country = Arc::new(StringArray::from(vec!["US", "CA", "US", "GB"])) as ArrayRef;
8878            cols.push(Arc::new(StructArray::new(fs, vec![street, zip, country], None)) as ArrayRef);
8879        }
8880        {
8881            let keys = StringArray::from(vec!["a", "b", "c", "neg", "pi", "ok"]);
8882            let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4, 4, 6]));
8883            let tid_s = 0; // string
8884            let tid_d = 1; // double
8885            let tid_n = 2; // null
8886            let type_ids = vec![tid_d, tid_n, tid_s, tid_d, tid_d, tid_s];
8887            let offsets = vec![0, 0, 0, 1, 2, 1];
8888            let pi_5dp = (std::f64::consts::PI * 100_000.0).trunc() / 100_000.0;
8889            let vals = mk_dense_union(&uf_map_vals, type_ids, offsets, |f| match f.data_type() {
8890                DataType::Float64 => {
8891                    Some(Arc::new(Float64Array::from(vec![1.5f64, -0.5, pi_5dp])) as ArrayRef)
8892                }
8893                DataType::Utf8 => {
8894                    Some(Arc::new(StringArray::from(vec!["yes", "true"])) as ArrayRef)
8895                }
8896                DataType::Null => Some(Arc::new(NullArray::new(1)) as ArrayRef),
8897                _ => None,
8898            });
8899            let entries = StructArray::new(
8900                Fields::from(vec![
8901                    Field::new("key", DataType::Utf8, false),
8902                    Field::new(
8903                        "value",
8904                        DataType::Union(uf_map_vals.clone(), UnionMode::Dense),
8905                        true,
8906                    ),
8907                ]),
8908                vec![Arc::new(keys) as ArrayRef, vals],
8909                None,
8910            );
8911            let map = Arc::new(MapArray::new(
8912                map_entries_field.clone(),
8913                moff,
8914                entries,
8915                None,
8916                false,
8917            )) as ArrayRef;
8918            cols.push(map);
8919        }
8920        {
8921            let type_ids = vec![
8922                2, 1, 0, 2, 0, 1, 2, 2, 1, 0,
8923                2, // long,string,null,long,null,string,long,long,string,null,long
8924            ];
8925            let offsets = vec![0, 0, 0, 1, 1, 1, 2, 3, 2, 2, 4];
8926            let values =
8927                mk_dense_union(&uf_arr_items, type_ids, offsets, |f| match f.data_type() {
8928                    DataType::Int64 => {
8929                        Some(Arc::new(Int64Array::from(vec![1i64, -3, 0, -1, 0])) as ArrayRef)
8930                    }
8931                    DataType::Utf8 => {
8932                        Some(Arc::new(StringArray::from(vec!["x", "z", "end"])) as ArrayRef)
8933                    }
8934                    DataType::Null => Some(Arc::new(NullArray::new(3)) as ArrayRef),
8935                    _ => None,
8936                });
8937            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 4, 7, 8, 11]));
8938            let arr = Arc::new(
8939                ListArray::try_new(arr_items_field.clone(), list_offsets, values, None).unwrap(),
8940            ) as ArrayRef;
8941            cols.push(arr);
8942        }
8943        {
8944            let keys = Int32Array::from(vec![1, 2, 3, 0]); // NEW, PROCESSING, DONE, UNKNOWN
8945            let values = Arc::new(StringArray::from(vec![
8946                "UNKNOWN",
8947                "NEW",
8948                "PROCESSING",
8949                "DONE",
8950            ])) as ArrayRef;
8951            let dict = DictionaryArray::<Int32Type>::try_new(keys, values).unwrap();
8952            cols.push(Arc::new(dict) as ArrayRef);
8953        }
8954        cols.push(Arc::new(IntervalMonthDayNanoArray::from(vec![
8955            dur_small, dur_zero, dur_large, dur_2years,
8956        ])) as ArrayRef);
8957        cols.push(Arc::new(TimestampMicrosecondArray::from(vec![
8958            ts_us_2024_01_01 + 123_456,
8959            0,
8960            ts_us_2024_01_01 + 101_112,
8961            987_654_321,
8962        ])) as ArrayRef);
8963        cols.push(Arc::new(TimestampMillisecondArray::from(vec![
8964            ts_ms_2024_01_01 + 86_400_000,
8965            0,
8966            ts_ms_2024_01_01 + 789,
8967            123_456_789,
8968        ])) as ArrayRef);
8969        {
8970            let a = TimestampMicrosecondArray::from(vec![
8971                ts_us_2024_01_01,
8972                1,
8973                ts_us_2024_01_01 + 456,
8974                0,
8975            ])
8976            .with_timezone("+00:00");
8977            cols.push(Arc::new(a) as ArrayRef);
8978        }
8979        {
8980            let a = TimestampMillisecondArray::from(vec![
8981                ts_ms_2024_01_01,
8982                -1,
8983                ts_ms_2024_01_01 + 123,
8984                0,
8985            ])
8986            .with_timezone("+00:00");
8987            cols.push(Arc::new(a) as ArrayRef);
8988        }
8989        cols.push(Arc::new(Time64MicrosecondArray::from(vec![
8990            time_us_eod,
8991            0,
8992            1,
8993            1_000_000,
8994        ])) as ArrayRef);
8995        cols.push(Arc::new(Time32MillisecondArray::from(vec![
8996            time_ms_a,
8997            0,
8998            1,
8999            86_400_000 - 1,
9000        ])) as ArrayRef);
9001        cols.push(Arc::new(Date32Array::from(vec![date_a, 0, 1, 365])) as ArrayRef);
9002        {
9003            let it = [Some(uuid1), Some(uuid2), Some(uuid1), Some(uuid2)].into_iter();
9004            cols.push(Arc::new(
9005                FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
9006            ) as ArrayRef);
9007        }
9008        {
9009            #[cfg(feature = "small_decimals")]
9010            let arr = Arc::new(
9011                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
9012                    .with_precision_and_scale(20, 4)
9013                    .unwrap(),
9014            ) as ArrayRef;
9015            #[cfg(not(feature = "small_decimals"))]
9016            let arr = Arc::new(
9017                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
9018                    .with_precision_and_scale(20, 4)
9019                    .unwrap(),
9020            ) as ArrayRef;
9021            cols.push(arr);
9022        }
9023        {
9024            #[cfg(feature = "small_decimals")]
9025            let arr = Arc::new(
9026                Decimal64Array::from_iter_values([123456i64, -1, 0, 9_999_999_999i64])
9027                    .with_precision_and_scale(10, 2)
9028                    .unwrap(),
9029            ) as ArrayRef;
9030            #[cfg(not(feature = "small_decimals"))]
9031            let arr = Arc::new(
9032                Decimal128Array::from_iter_values([123456i128, -1, 0, 9_999_999_999i128])
9033                    .with_precision_and_scale(10, 2)
9034                    .unwrap(),
9035            ) as ArrayRef;
9036            cols.push(arr);
9037        }
9038        {
9039            let it = [
9040                Some(*b"0123456789ABCDEF"),
9041                Some([0u8; 16]),
9042                Some(*b"ABCDEFGHIJKLMNOP"),
9043                Some([0xAA; 16]),
9044            ]
9045            .into_iter();
9046            cols.push(Arc::new(
9047                FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
9048            ) as ArrayRef);
9049        }
9050        cols.push(Arc::new(BinaryArray::from(vec![
9051            b"\x00\x01".as_ref(),
9052            b"".as_ref(),
9053            b"\xFF\x00".as_ref(),
9054            b"\x10\x20\x30\x40".as_ref(),
9055        ])) as ArrayRef);
9056        cols.push(Arc::new(StringArray::from(vec!["hello", "", "world", "✓ unicode"])) as ArrayRef);
9057        {
9058            let tids = vec![0, 1, 2, 1];
9059            let offs = vec![0, 0, 0, 1];
9060            let arr = mk_dense_union(&uf_tri, tids, offs, |f| match f.data_type() {
9061                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![0])) as ArrayRef),
9062                DataType::Utf8 => Some(Arc::new(StringArray::from(vec!["hi", ""])) as ArrayRef),
9063                DataType::Boolean => Some(Arc::new(BooleanArray::from(vec![true])) as ArrayRef),
9064                _ => None,
9065            });
9066            cols.push(arr);
9067        }
9068        cols.push(Arc::new(StringArray::from(vec![
9069            Some("alpha"),
9070            None,
9071            Some("s3"),
9072            Some(""),
9073        ])) as ArrayRef);
9074        cols.push(Arc::new(Int32Array::from(vec![None, Some(42), None, Some(0)])) as ArrayRef);
9075        cols.push(Arc::new(Int64Array::from(vec![
9076            7_000_000_000i64,
9077            -2,
9078            0,
9079            -9_876_543_210i64,
9080        ])) as ArrayRef);
9081        cols.push(Arc::new(Int64Array::from(vec![7i64, -1, 0, 123])) as ArrayRef);
9082        cols.push(Arc::new(Float64Array::from(vec![2.5f64, -1.0, 7.0, -2.25])) as ArrayRef);
9083        cols.push(Arc::new(Float64Array::from(vec![1.25f64, -0.0, 3.5, 9.75])) as ArrayRef);
9084        cols.push(Arc::new(BooleanArray::from(vec![true, false, true, false])) as ArrayRef);
9085        cols.push(Arc::new(Int64Array::from(vec![1, 2, 3, 4])) as ArrayRef);
9086        let expected = RecordBatch::try_new(expected_schema, cols).unwrap();
9087        assert_eq!(
9088            expected, batch,
9089            "entire RecordBatch mismatch (schema, all columns, all rows)"
9090        );
9091    }
9092
9093    #[test]
9094    fn test_bad_varint_bug_nullable_array_items() {
9095        use flate2::read::GzDecoder;
9096        use std::io::Read;
9097        let manifest_dir = env!("CARGO_MANIFEST_DIR");
9098        let gz_path = format!("{manifest_dir}/test/data/bad-varint-bug.avro.gz");
9099        let gz_file = File::open(&gz_path).expect("test file should exist");
9100        let mut decoder = GzDecoder::new(gz_file);
9101        let mut avro_bytes = Vec::new();
9102        decoder
9103            .read_to_end(&mut avro_bytes)
9104            .expect("should decompress");
9105        let reader_arrow_schema = Schema::new(vec![Field::new(
9106            "int_array",
9107            DataType::List(Arc::new(Field::new("element", DataType::Int32, true))),
9108            true,
9109        )])
9110        .with_metadata(HashMap::from([("avro.name".into(), "table".into())]));
9111        let reader_schema = AvroSchema::try_from(&reader_arrow_schema)
9112            .expect("should convert Arrow schema to Avro");
9113        let mut reader = ReaderBuilder::new()
9114            .with_reader_schema(reader_schema)
9115            .build(Cursor::new(avro_bytes))
9116            .expect("should build reader");
9117        let batch = reader
9118            .next()
9119            .expect("should have one batch")
9120            .expect("reading should succeed without bad varint error");
9121        assert_eq!(batch.num_rows(), 1);
9122        let list_col = batch
9123            .column(0)
9124            .as_any()
9125            .downcast_ref::<ListArray>()
9126            .expect("should be ListArray");
9127        assert_eq!(list_col.len(), 1);
9128        let values = list_col.values();
9129        let int_values = values.as_primitive::<Int32Type>();
9130        assert_eq!(int_values.len(), 2);
9131        assert_eq!(int_values.value(0), 1);
9132        assert_eq!(int_values.value(1), 2);
9133    }
9134}