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::header::HeaderDecoder;
1405    use crate::reader::record::RecordDecoder;
1406    use crate::reader::{Decoder, Reader, ReaderBuilder};
1407    use crate::schema::{
1408        AVRO_ENUM_SYMBOLS_METADATA_KEY, AVRO_NAME_METADATA_KEY, AVRO_NAMESPACE_METADATA_KEY,
1409        AvroSchema, CONFLUENT_MAGIC, Fingerprint, FingerprintAlgorithm, PrimitiveType,
1410        SINGLE_OBJECT_MAGIC, SchemaStore,
1411    };
1412    use crate::test_util::arrow_test_data;
1413    use crate::writer::AvroWriter;
1414    use arrow_array::builder::{
1415        ArrayBuilder, BooleanBuilder, Float32Builder, Int32Builder, Int64Builder, ListBuilder,
1416        MapBuilder, StringBuilder, StructBuilder,
1417    };
1418    #[cfg(feature = "snappy")]
1419    use arrow_array::builder::{Float64Builder, MapFieldNames};
1420    use arrow_array::cast::AsArray;
1421    #[cfg(not(feature = "avro_custom_types"))]
1422    use arrow_array::types::Int64Type;
1423    #[cfg(feature = "avro_custom_types")]
1424    use arrow_array::types::{
1425        DurationMicrosecondType, DurationMillisecondType, DurationNanosecondType,
1426        DurationSecondType,
1427    };
1428    use arrow_array::types::{Int32Type, IntervalMonthDayNanoType};
1429    use arrow_array::*;
1430    #[cfg(feature = "snappy")]
1431    use arrow_buffer::{Buffer, NullBuffer};
1432    use arrow_buffer::{IntervalMonthDayNano, OffsetBuffer, ScalarBuffer, i256};
1433    #[cfg(feature = "avro_custom_types")]
1434    use arrow_schema::{
1435        ArrowError, DataType, Field, FieldRef, Fields, IntervalUnit, Schema, TimeUnit, UnionFields,
1436        UnionMode,
1437    };
1438    #[cfg(not(feature = "avro_custom_types"))]
1439    use arrow_schema::{
1440        ArrowError, DataType, Field, FieldRef, Fields, IntervalUnit, Schema, UnionFields, UnionMode,
1441    };
1442    use bytes::Bytes;
1443    use futures::executor::block_on;
1444    use futures::{Stream, StreamExt, TryStreamExt, stream};
1445    use serde_json::{Value, json};
1446    use std::collections::HashMap;
1447    use std::fs::File;
1448    use std::io::{BufReader, Cursor};
1449    use std::sync::Arc;
1450
1451    fn files() -> impl Iterator<Item = &'static str> {
1452        [
1453            // TODO: avoid requiring snappy for this file
1454            #[cfg(feature = "snappy")]
1455            "avro/alltypes_plain.avro",
1456            #[cfg(feature = "snappy")]
1457            "avro/alltypes_plain.snappy.avro",
1458            #[cfg(feature = "zstd")]
1459            "avro/alltypes_plain.zstandard.avro",
1460            #[cfg(feature = "bzip2")]
1461            "avro/alltypes_plain.bzip2.avro",
1462            #[cfg(feature = "xz")]
1463            "avro/alltypes_plain.xz.avro",
1464        ]
1465        .into_iter()
1466    }
1467
1468    fn read_file(path: &str, batch_size: usize, utf8_view: bool) -> RecordBatch {
1469        let file = File::open(path).unwrap();
1470        let reader = ReaderBuilder::new()
1471            .with_batch_size(batch_size)
1472            .with_utf8_view(utf8_view)
1473            .build(BufReader::new(file))
1474            .unwrap();
1475        let schema = reader.schema();
1476        let batches = reader.collect::<Result<Vec<_>, _>>().unwrap();
1477        arrow::compute::concat_batches(&schema, &batches).unwrap()
1478    }
1479
1480    fn read_file_strict(
1481        path: &str,
1482        batch_size: usize,
1483        utf8_view: bool,
1484    ) -> Result<Reader<BufReader<File>>, ArrowError> {
1485        let file = File::open(path)?;
1486        ReaderBuilder::new()
1487            .with_batch_size(batch_size)
1488            .with_utf8_view(utf8_view)
1489            .with_strict_mode(true)
1490            .build(BufReader::new(file))
1491    }
1492
1493    fn decode_stream<S: Stream<Item = Bytes> + Unpin>(
1494        mut decoder: Decoder,
1495        mut input: S,
1496    ) -> impl Stream<Item = Result<RecordBatch, ArrowError>> {
1497        async_stream::try_stream! {
1498            if let Some(data) = input.next().await {
1499                let consumed = decoder.decode(&data)?;
1500                if consumed < data.len() {
1501                    Err(ArrowError::ParseError(
1502                        "did not consume all bytes".to_string(),
1503                    ))?;
1504                }
1505            }
1506            if let Some(batch) = decoder.flush()? {
1507                yield batch
1508            }
1509        }
1510    }
1511
1512    fn make_record_schema(pt: PrimitiveType) -> AvroSchema {
1513        let js = format!(
1514            r#"{{"type":"record","name":"TestRecord","fields":[{{"name":"a","type":"{}"}}]}}"#,
1515            pt.as_ref()
1516        );
1517        AvroSchema::new(js)
1518    }
1519
1520    fn make_two_schema_store() -> (
1521        SchemaStore,
1522        Fingerprint,
1523        Fingerprint,
1524        AvroSchema,
1525        AvroSchema,
1526    ) {
1527        let schema_int = make_record_schema(PrimitiveType::Int);
1528        let schema_long = make_record_schema(PrimitiveType::Long);
1529        let mut store = SchemaStore::new();
1530        let fp_int = store
1531            .register(schema_int.clone())
1532            .expect("register int schema");
1533        let fp_long = store
1534            .register(schema_long.clone())
1535            .expect("register long schema");
1536        (store, fp_int, fp_long, schema_int, schema_long)
1537    }
1538
1539    fn make_prefix(fp: Fingerprint) -> Vec<u8> {
1540        match fp {
1541            Fingerprint::Rabin(v) => {
1542                let mut out = Vec::with_capacity(2 + 8);
1543                out.extend_from_slice(&SINGLE_OBJECT_MAGIC);
1544                out.extend_from_slice(&v.to_le_bytes());
1545                out
1546            }
1547            Fingerprint::Id(v) => {
1548                panic!("make_prefix expects a Rabin fingerprint, got ({v})");
1549            }
1550            Fingerprint::Id64(v) => {
1551                panic!("make_prefix expects a Rabin fingerprint, got ({v})");
1552            }
1553            #[cfg(feature = "md5")]
1554            Fingerprint::MD5(v) => {
1555                panic!("make_prefix expects a Rabin fingerprint, got ({v:?})");
1556            }
1557            #[cfg(feature = "sha256")]
1558            Fingerprint::SHA256(id) => {
1559                panic!("make_prefix expects a Rabin fingerprint, got ({id:?})");
1560            }
1561        }
1562    }
1563
1564    fn make_decoder(store: &SchemaStore, fp: Fingerprint, reader_schema: &AvroSchema) -> Decoder {
1565        ReaderBuilder::new()
1566            .with_batch_size(8)
1567            .with_reader_schema(reader_schema.clone())
1568            .with_writer_schema_store(store.clone())
1569            .with_active_fingerprint(fp)
1570            .build_decoder()
1571            .expect("decoder")
1572    }
1573
1574    fn make_id_prefix(id: u32, additional: usize) -> Vec<u8> {
1575        let capacity = CONFLUENT_MAGIC.len() + size_of::<u32>() + additional;
1576        let mut out = Vec::with_capacity(capacity);
1577        out.extend_from_slice(&CONFLUENT_MAGIC);
1578        out.extend_from_slice(&id.to_be_bytes());
1579        out
1580    }
1581
1582    fn make_message_id(id: u32, value: i64) -> Vec<u8> {
1583        let encoded_value = encode_zigzag(value);
1584        let mut msg = make_id_prefix(id, encoded_value.len());
1585        msg.extend_from_slice(&encoded_value);
1586        msg
1587    }
1588
1589    fn make_id64_prefix(id: u64, additional: usize) -> Vec<u8> {
1590        let capacity = CONFLUENT_MAGIC.len() + size_of::<u64>() + additional;
1591        let mut out = Vec::with_capacity(capacity);
1592        out.extend_from_slice(&CONFLUENT_MAGIC);
1593        out.extend_from_slice(&id.to_be_bytes());
1594        out
1595    }
1596
1597    fn make_message_id64(id: u64, value: i64) -> Vec<u8> {
1598        let encoded_value = encode_zigzag(value);
1599        let mut msg = make_id64_prefix(id, encoded_value.len());
1600        msg.extend_from_slice(&encoded_value);
1601        msg
1602    }
1603
1604    fn make_value_schema(pt: PrimitiveType) -> AvroSchema {
1605        let json_schema = format!(
1606            r#"{{"type":"record","name":"S","fields":[{{"name":"v","type":"{}"}}]}}"#,
1607            pt.as_ref()
1608        );
1609        AvroSchema::new(json_schema)
1610    }
1611
1612    fn encode_zigzag(value: i64) -> Vec<u8> {
1613        let mut n = ((value << 1) ^ (value >> 63)) as u64;
1614        let mut out = Vec::new();
1615        loop {
1616            if (n & !0x7F) == 0 {
1617                out.push(n as u8);
1618                break;
1619            } else {
1620                out.push(((n & 0x7F) | 0x80) as u8);
1621                n >>= 7;
1622            }
1623        }
1624        out
1625    }
1626
1627    fn make_message(fp: Fingerprint, value: i64) -> Vec<u8> {
1628        let mut msg = make_prefix(fp);
1629        msg.extend_from_slice(&encode_zigzag(value));
1630        msg
1631    }
1632
1633    fn load_writer_schema_json(path: &str) -> Value {
1634        let file = File::open(path).unwrap();
1635        let header = super::read_header(BufReader::new(file)).unwrap();
1636        let schema = header.schema().unwrap().unwrap();
1637        serde_json::to_value(&schema).unwrap()
1638    }
1639
1640    fn make_reader_schema_with_promotions(
1641        path: &str,
1642        promotions: &HashMap<&str, &str>,
1643    ) -> AvroSchema {
1644        let mut root = load_writer_schema_json(path);
1645        assert_eq!(root["type"], "record", "writer schema must be a record");
1646        let fields = root
1647            .get_mut("fields")
1648            .and_then(|f| f.as_array_mut())
1649            .expect("record has fields");
1650        for f in fields.iter_mut() {
1651            let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
1652                continue;
1653            };
1654            if let Some(new_ty) = promotions.get(name) {
1655                let ty = f.get_mut("type").expect("field has a type");
1656                match ty {
1657                    Value::String(_) => {
1658                        *ty = Value::String((*new_ty).to_string());
1659                    }
1660                    // Union
1661                    Value::Array(arr) => {
1662                        for b in arr.iter_mut() {
1663                            match b {
1664                                Value::String(s) if s != "null" => {
1665                                    *b = Value::String((*new_ty).to_string());
1666                                    break;
1667                                }
1668                                Value::Object(_) => {
1669                                    *b = Value::String((*new_ty).to_string());
1670                                    break;
1671                                }
1672                                _ => {}
1673                            }
1674                        }
1675                    }
1676                    Value::Object(_) => {
1677                        *ty = Value::String((*new_ty).to_string());
1678                    }
1679                    _ => {}
1680                }
1681            }
1682        }
1683        AvroSchema::new(root.to_string())
1684    }
1685
1686    fn make_reader_schema_with_enum_remap(
1687        path: &str,
1688        remap: &HashMap<&str, Vec<&str>>,
1689    ) -> AvroSchema {
1690        let mut root = load_writer_schema_json(path);
1691        assert_eq!(root["type"], "record", "writer schema must be a record");
1692        let fields = root
1693            .get_mut("fields")
1694            .and_then(|f| f.as_array_mut())
1695            .expect("record has fields");
1696
1697        fn to_symbols_array(symbols: &[&str]) -> Value {
1698            Value::Array(symbols.iter().map(|s| Value::String((*s).into())).collect())
1699        }
1700
1701        fn update_enum_symbols(ty: &mut Value, symbols: &Value) {
1702            match ty {
1703                Value::Object(map) => {
1704                    if matches!(map.get("type"), Some(Value::String(t)) if t == "enum") {
1705                        map.insert("symbols".to_string(), symbols.clone());
1706                    }
1707                }
1708                Value::Array(arr) => {
1709                    for b in arr.iter_mut() {
1710                        if let Value::Object(map) = b {
1711                            if matches!(map.get("type"), Some(Value::String(t)) if t == "enum") {
1712                                map.insert("symbols".to_string(), symbols.clone());
1713                            }
1714                        }
1715                    }
1716                }
1717                _ => {}
1718            }
1719        }
1720        for f in fields.iter_mut() {
1721            let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
1722                continue;
1723            };
1724            if let Some(new_symbols) = remap.get(name) {
1725                let symbols_val = to_symbols_array(new_symbols);
1726                let ty = f.get_mut("type").expect("field has a type");
1727                update_enum_symbols(ty, &symbols_val);
1728            }
1729        }
1730        AvroSchema::new(root.to_string())
1731    }
1732
1733    fn read_alltypes_with_reader_schema(path: &str, reader_schema: AvroSchema) -> RecordBatch {
1734        let file = File::open(path).unwrap();
1735        let reader = ReaderBuilder::new()
1736            .with_batch_size(1024)
1737            .with_utf8_view(false)
1738            .with_reader_schema(reader_schema)
1739            .build(BufReader::new(file))
1740            .unwrap();
1741        let schema = reader.schema();
1742        let batches = reader.collect::<Result<Vec<_>, _>>().unwrap();
1743        arrow::compute::concat_batches(&schema, &batches).unwrap()
1744    }
1745
1746    fn make_reader_schema_with_selected_fields_in_order(
1747        path: &str,
1748        selected: &[&str],
1749    ) -> AvroSchema {
1750        let mut root = load_writer_schema_json(path);
1751        assert_eq!(root["type"], "record", "writer schema must be a record");
1752        let writer_fields = root
1753            .get("fields")
1754            .and_then(|f| f.as_array())
1755            .expect("record has fields");
1756        let mut field_map: HashMap<String, Value> = HashMap::with_capacity(writer_fields.len());
1757        for f in writer_fields {
1758            if let Some(name) = f.get("name").and_then(|n| n.as_str()) {
1759                field_map.insert(name.to_string(), f.clone());
1760            }
1761        }
1762        let mut new_fields = Vec::with_capacity(selected.len());
1763        for name in selected {
1764            let f = field_map
1765                .get(*name)
1766                .unwrap_or_else(|| panic!("field '{name}' not found in writer schema"))
1767                .clone();
1768            new_fields.push(f);
1769        }
1770        root["fields"] = Value::Array(new_fields);
1771        AvroSchema::new(root.to_string())
1772    }
1773
1774    fn write_ocf(schema: &Schema, batches: &[RecordBatch]) -> Vec<u8> {
1775        let mut w = AvroWriter::new(Vec::<u8>::new(), schema.clone()).expect("writer");
1776        for b in batches {
1777            w.write(b).expect("write");
1778        }
1779        w.finish().expect("finish");
1780        w.into_inner()
1781    }
1782
1783    #[test]
1784    fn ocf_projection_no_reader_schema_reorder() -> Result<(), Box<dyn std::error::Error>> {
1785        // Writer: { id: int, name: string, is_active: boolean }
1786        let writer_schema = Schema::new(vec![
1787            Field::new("id", DataType::Int32, false),
1788            Field::new("name", DataType::Utf8, false),
1789            Field::new("is_active", DataType::Boolean, false),
1790        ]);
1791        let batch = RecordBatch::try_new(
1792            Arc::new(writer_schema.clone()),
1793            vec![
1794                Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef,
1795                Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
1796                Arc::new(BooleanArray::from(vec![true, false])) as ArrayRef,
1797            ],
1798        )?;
1799        let bytes = write_ocf(&writer_schema, &[batch]);
1800        // Project and reorder: [is_active, id]
1801        let mut reader = ReaderBuilder::new()
1802            .with_projection(vec![2, 0])
1803            .build(Cursor::new(bytes))?;
1804        let out = reader.next().unwrap()?;
1805        assert_eq!(out.num_columns(), 2);
1806        assert_eq!(out.schema().field(0).name(), "is_active");
1807        assert_eq!(out.schema().field(1).name(), "id");
1808        let is_active = out.column(0).as_boolean();
1809        assert!(is_active.value(0));
1810        assert!(!is_active.value(1));
1811        let id = out.column(1).as_primitive::<Int32Type>();
1812        assert_eq!(id.value(0), 1);
1813        assert_eq!(id.value(1), 2);
1814        Ok(())
1815    }
1816
1817    #[test]
1818    fn ocf_projection_with_reader_schema_alias_and_default()
1819    -> Result<(), Box<dyn std::error::Error>> {
1820        // Writer: { id: long, name: string }
1821        let writer_schema = Schema::new(vec![
1822            Field::new("id", DataType::Int64, false),
1823            Field::new("name", DataType::Utf8, false),
1824        ]);
1825        let batch = RecordBatch::try_new(
1826            Arc::new(writer_schema.clone()),
1827            vec![
1828                Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef,
1829                Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
1830            ],
1831        )?;
1832        let bytes = write_ocf(&writer_schema, &[batch]);
1833        // Reader adds alias + default field:
1834        //  - rename `name` -> `full_name` via aliases
1835        //  - add `is_active` with default true
1836        let reader_json = r#"
1837    {
1838      "type": "record",
1839      "name": "topLevelRecord",
1840      "fields": [
1841        { "name": "id", "type": "long" },
1842        { "name": "full_name", "type": ["null","string"], "aliases": ["name"], "default": null },
1843        { "name": "is_active", "type": "boolean", "default": true }
1844      ]
1845    }"#;
1846        // Project only [full_name, is_active] (indices relative to the reader schema)
1847        let mut reader = ReaderBuilder::new()
1848            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
1849            .with_projection(vec![1, 2])
1850            .build(Cursor::new(bytes))?;
1851        let out = reader.next().unwrap()?;
1852        assert_eq!(out.num_columns(), 2);
1853        assert_eq!(out.schema().field(0).name(), "full_name");
1854        assert_eq!(out.schema().field(1).name(), "is_active");
1855        let full_name = out.column(0).as_string::<i32>();
1856        assert_eq!(full_name.value(0), "a");
1857        assert_eq!(full_name.value(1), "b");
1858        let is_active = out.column(1).as_boolean();
1859        assert!(is_active.value(0));
1860        assert!(is_active.value(1));
1861        Ok(())
1862    }
1863
1864    #[test]
1865    fn projection_errors_out_of_bounds_and_duplicate() -> Result<(), Box<dyn std::error::Error>> {
1866        let writer_schema = Schema::new(vec![
1867            Field::new("a", DataType::Int32, false),
1868            Field::new("b", DataType::Int32, false),
1869        ]);
1870        let batch = RecordBatch::try_new(
1871            Arc::new(writer_schema.clone()),
1872            vec![
1873                Arc::new(Int32Array::from(vec![1])) as ArrayRef,
1874                Arc::new(Int32Array::from(vec![2])) as ArrayRef,
1875            ],
1876        )?;
1877        let bytes = write_ocf(&writer_schema, &[batch]);
1878        let err = ReaderBuilder::new()
1879            .with_projection(vec![2])
1880            .build(Cursor::new(bytes.clone()))
1881            .unwrap_err();
1882        assert!(matches!(err, ArrowError::AvroError(_)));
1883        assert!(err.to_string().contains("out of bounds"));
1884        let err = ReaderBuilder::new()
1885            .with_projection(vec![0, 0])
1886            .build(Cursor::new(bytes))
1887            .unwrap_err();
1888        assert!(matches!(err, ArrowError::AvroError(_)));
1889        assert!(err.to_string().contains("Duplicate projection index"));
1890        Ok(())
1891    }
1892
1893    #[test]
1894    #[cfg(feature = "snappy")]
1895    fn test_alltypes_plain_with_projection_and_reader_schema() {
1896        use std::fs::File;
1897        use std::io::BufReader;
1898        let path = arrow_test_data("avro/alltypes_plain.avro");
1899        // Build a reader schema that selects [double_col, id, tinyint_col] in that order
1900        let reader_schema = make_reader_schema_with_selected_fields_in_order(
1901            &path,
1902            &["double_col", "id", "tinyint_col"],
1903        );
1904        let file = File::open(&path).expect("open avro/alltypes_plain.avro");
1905        let reader = ReaderBuilder::new()
1906            .with_batch_size(1024)
1907            .with_reader_schema(reader_schema)
1908            .with_projection(vec![1, 2]) // Select indices 1 and 2 from reader schema: [id, tinyint_col]
1909            .build(BufReader::new(file))
1910            .expect("build reader with projection and reader schema");
1911        let schema = reader.schema();
1912        // Verify the projected schema has exactly 2 fields in the correct order
1913        assert_eq!(schema.fields().len(), 2);
1914        assert_eq!(schema.field(0).name(), "id");
1915        assert_eq!(schema.field(1).name(), "tinyint_col");
1916        let batches: Vec<RecordBatch> = reader.collect::<Result<Vec<_>, _>>().unwrap();
1917        assert_eq!(batches.len(), 1);
1918        let batch = &batches[0];
1919        assert_eq!(batch.num_rows(), 8);
1920        assert_eq!(batch.num_columns(), 2);
1921        // Build expected batch with exact values from alltypes_plain.avro:
1922        // - id values: [4, 5, 6, 7, 2, 3, 0, 1]
1923        // - tinyint_col values: [0, 1, 0, 1, 0, 1, 0, 1] (i.e., row_index % 2)
1924        let expected = RecordBatch::try_from_iter_with_nullable([
1925            (
1926                "id",
1927                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as ArrayRef,
1928                true,
1929            ),
1930            (
1931                "tinyint_col",
1932                Arc::new(Int32Array::from(vec![0, 1, 0, 1, 0, 1, 0, 1])) as ArrayRef,
1933                true,
1934            ),
1935        ])
1936        .unwrap();
1937        assert_eq!(
1938            batch, &expected,
1939            "Projected batch mismatch for alltypes_plain.avro with reader schema and projection [1, 2]"
1940        );
1941    }
1942
1943    #[test]
1944    #[cfg(feature = "snappy")]
1945    fn test_alltypes_plain_with_projection() {
1946        use std::fs::File;
1947        use std::io::BufReader;
1948        let path = arrow_test_data("avro/alltypes_plain.avro");
1949        let file = File::open(&path).expect("open avro/alltypes_plain.avro");
1950        let reader = ReaderBuilder::new()
1951            .with_batch_size(1024)
1952            .with_projection(vec![2, 0, 5])
1953            .build(BufReader::new(file))
1954            .expect("build reader with projection");
1955        let schema = reader.schema();
1956        assert_eq!(schema.fields().len(), 3);
1957        assert_eq!(schema.field(0).name(), "tinyint_col");
1958        assert_eq!(schema.field(1).name(), "id");
1959        assert_eq!(schema.field(2).name(), "bigint_col");
1960        let batches: Vec<RecordBatch> = reader.collect::<Result<Vec<_>, _>>().unwrap();
1961        assert_eq!(batches.len(), 1);
1962        let batch = &batches[0];
1963        assert_eq!(batch.num_rows(), 8);
1964        assert_eq!(batch.num_columns(), 3);
1965        let expected = RecordBatch::try_from_iter_with_nullable([
1966            (
1967                "tinyint_col",
1968                Arc::new(Int32Array::from(vec![0, 1, 0, 1, 0, 1, 0, 1])) as ArrayRef,
1969                true,
1970            ),
1971            (
1972                "id",
1973                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as ArrayRef,
1974                true,
1975            ),
1976            (
1977                "bigint_col",
1978                Arc::new(Int64Array::from(vec![0, 10, 0, 10, 0, 10, 0, 10])) as ArrayRef,
1979                true,
1980            ),
1981        ])
1982        .unwrap();
1983        assert_eq!(
1984            batch, &expected,
1985            "Projected batch mismatch for alltypes_plain.avro with projection [2, 0, 5]"
1986        );
1987    }
1988
1989    #[test]
1990    fn writer_string_reader_nullable_with_alias() -> Result<(), Box<dyn std::error::Error>> {
1991        let writer_schema = Schema::new(vec![
1992            Field::new("id", DataType::Int64, false),
1993            Field::new("name", DataType::Utf8, false),
1994        ]);
1995        let batch = RecordBatch::try_new(
1996            Arc::new(writer_schema.clone()),
1997            vec![
1998                Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef,
1999                Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
2000            ],
2001        )?;
2002        let bytes = write_ocf(&writer_schema, &[batch]);
2003        let reader_json = r#"
2004    {
2005      "type": "record",
2006      "name": "topLevelRecord",
2007      "fields": [
2008        { "name": "id", "type": "long" },
2009        { "name": "full_name", "type": ["null","string"], "aliases": ["name"], "default": null },
2010        { "name": "is_active", "type": "boolean", "default": true }
2011      ]
2012    }"#;
2013        let mut reader = ReaderBuilder::new()
2014            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
2015            .build(Cursor::new(bytes))?;
2016        let out = reader.next().unwrap()?;
2017        let full_name = out.column(1).as_string::<i32>();
2018        assert_eq!(full_name.value(0), "a");
2019        assert_eq!(full_name.value(1), "b");
2020        Ok(())
2021    }
2022
2023    #[test]
2024    fn writer_string_reader_string_null_order_second() -> Result<(), Box<dyn std::error::Error>> {
2025        // Writer: { name: string }
2026        let writer_schema = Schema::new(vec![Field::new("name", DataType::Utf8, false)]);
2027        let batch = RecordBatch::try_new(
2028            Arc::new(writer_schema.clone()),
2029            vec![Arc::new(StringArray::from(vec!["x", "y"])) as ArrayRef],
2030        )?;
2031        let bytes = write_ocf(&writer_schema, &[batch]);
2032
2033        // Reader: ["string","null"] (NullSecond)
2034        let reader_json = r#"
2035    {
2036      "type":"record", "name":"topLevelRecord",
2037      "fields":[ { "name":"name", "type":["string","null"], "default":"x" } ]
2038    }"#;
2039
2040        let mut reader = ReaderBuilder::new()
2041            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
2042            .build(Cursor::new(bytes))?;
2043
2044        let out = reader.next().unwrap()?;
2045        assert_eq!(out.num_rows(), 2);
2046
2047        // Should decode as non-null strings (writer non-union -> reader union)
2048        let name = out.column(0).as_string::<i32>();
2049        assert_eq!(name.value(0), "x");
2050        assert_eq!(name.value(1), "y");
2051
2052        Ok(())
2053    }
2054
2055    #[test]
2056    fn promotion_writer_int_reader_nullable_long() -> Result<(), Box<dyn std::error::Error>> {
2057        // Writer: { v: int }
2058        let writer_schema = Schema::new(vec![Field::new("v", DataType::Int32, false)]);
2059        let batch = RecordBatch::try_new(
2060            Arc::new(writer_schema.clone()),
2061            vec![Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef],
2062        )?;
2063        let bytes = write_ocf(&writer_schema, &[batch]);
2064
2065        // Reader: { v: ["null","long"] }
2066        let reader_json = r#"
2067    {
2068      "type":"record", "name":"topLevelRecord",
2069      "fields":[ { "name":"v", "type":["null","long"], "default": null } ]
2070    }"#;
2071
2072        let mut reader = ReaderBuilder::new()
2073            .with_reader_schema(AvroSchema::new(reader_json.to_string()))
2074            .build(Cursor::new(bytes))?;
2075
2076        let out = reader.next().unwrap()?;
2077        assert_eq!(out.num_rows(), 3);
2078
2079        // Should have promoted to Int64 and be non-null (no union tag in writer)
2080        let v = out
2081            .column(0)
2082            .as_primitive::<arrow_array::types::Int64Type>();
2083        assert_eq!(v.values(), &[1, 2, 3]);
2084        assert!(
2085            out.column(0).nulls().is_none(),
2086            "expected no validity bitmap for all-valid column"
2087        );
2088
2089        Ok(())
2090    }
2091
2092    #[test]
2093    fn test_alltypes_schema_promotion_mixed() {
2094        for file in files() {
2095            let file = arrow_test_data(file);
2096            let mut promotions: HashMap<&str, &str> = HashMap::new();
2097            promotions.insert("id", "long");
2098            promotions.insert("tinyint_col", "float");
2099            promotions.insert("smallint_col", "double");
2100            promotions.insert("int_col", "double");
2101            promotions.insert("bigint_col", "double");
2102            promotions.insert("float_col", "double");
2103            promotions.insert("date_string_col", "string");
2104            promotions.insert("string_col", "string");
2105            let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2106            let batch = read_alltypes_with_reader_schema(&file, reader_schema);
2107            let expected = RecordBatch::try_from_iter_with_nullable([
2108                (
2109                    "id",
2110                    Arc::new(Int64Array::from(vec![4i64, 5, 6, 7, 2, 3, 0, 1])) as _,
2111                    true,
2112                ),
2113                (
2114                    "bool_col",
2115                    Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
2116                    true,
2117                ),
2118                (
2119                    "tinyint_col",
2120                    Arc::new(Float32Array::from_iter_values(
2121                        (0..8).map(|x| (x % 2) as f32),
2122                    )) as _,
2123                    true,
2124                ),
2125                (
2126                    "smallint_col",
2127                    Arc::new(Float64Array::from_iter_values(
2128                        (0..8).map(|x| (x % 2) as f64),
2129                    )) as _,
2130                    true,
2131                ),
2132                (
2133                    "int_col",
2134                    Arc::new(Float64Array::from_iter_values(
2135                        (0..8).map(|x| (x % 2) as f64),
2136                    )) as _,
2137                    true,
2138                ),
2139                (
2140                    "bigint_col",
2141                    Arc::new(Float64Array::from_iter_values(
2142                        (0..8).map(|x| ((x % 2) * 10) as f64),
2143                    )) as _,
2144                    true,
2145                ),
2146                (
2147                    "float_col",
2148                    Arc::new(Float64Array::from_iter_values(
2149                        (0..8).map(|x| ((x % 2) as f32 * 1.1f32) as f64),
2150                    )) as _,
2151                    true,
2152                ),
2153                (
2154                    "double_col",
2155                    Arc::new(Float64Array::from_iter_values(
2156                        (0..8).map(|x| (x % 2) as f64 * 10.1),
2157                    )) as _,
2158                    true,
2159                ),
2160                (
2161                    "date_string_col",
2162                    Arc::new(StringArray::from(vec![
2163                        "03/01/09", "03/01/09", "04/01/09", "04/01/09", "02/01/09", "02/01/09",
2164                        "01/01/09", "01/01/09",
2165                    ])) as _,
2166                    true,
2167                ),
2168                (
2169                    "string_col",
2170                    Arc::new(StringArray::from(
2171                        (0..8)
2172                            .map(|x| if x % 2 == 0 { "0" } else { "1" })
2173                            .collect::<Vec<_>>(),
2174                    )) as _,
2175                    true,
2176                ),
2177                (
2178                    "timestamp_col",
2179                    Arc::new(
2180                        TimestampMicrosecondArray::from_iter_values([
2181                            1235865600000000, // 2009-03-01T00:00:00.000
2182                            1235865660000000, // 2009-03-01T00:01:00.000
2183                            1238544000000000, // 2009-04-01T00:00:00.000
2184                            1238544060000000, // 2009-04-01T00:01:00.000
2185                            1233446400000000, // 2009-02-01T00:00:00.000
2186                            1233446460000000, // 2009-02-01T00:01:00.000
2187                            1230768000000000, // 2009-01-01T00:00:00.000
2188                            1230768060000000, // 2009-01-01T00:01:00.000
2189                        ])
2190                        .with_timezone("+00:00"),
2191                    ) as _,
2192                    true,
2193                ),
2194            ])
2195            .unwrap();
2196            assert_eq!(batch, expected, "mismatch for file {file}");
2197        }
2198    }
2199
2200    #[test]
2201    fn test_alltypes_schema_promotion_long_to_float_only() {
2202        for file in files() {
2203            let file = arrow_test_data(file);
2204            let mut promotions: HashMap<&str, &str> = HashMap::new();
2205            promotions.insert("bigint_col", "float");
2206            let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2207            let batch = read_alltypes_with_reader_schema(&file, reader_schema);
2208            let expected = RecordBatch::try_from_iter_with_nullable([
2209                (
2210                    "id",
2211                    Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
2212                    true,
2213                ),
2214                (
2215                    "bool_col",
2216                    Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
2217                    true,
2218                ),
2219                (
2220                    "tinyint_col",
2221                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2222                    true,
2223                ),
2224                (
2225                    "smallint_col",
2226                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2227                    true,
2228                ),
2229                (
2230                    "int_col",
2231                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2232                    true,
2233                ),
2234                (
2235                    "bigint_col",
2236                    Arc::new(Float32Array::from_iter_values(
2237                        (0..8).map(|x| ((x % 2) * 10) as f32),
2238                    )) as _,
2239                    true,
2240                ),
2241                (
2242                    "float_col",
2243                    Arc::new(Float32Array::from_iter_values(
2244                        (0..8).map(|x| (x % 2) as f32 * 1.1),
2245                    )) as _,
2246                    true,
2247                ),
2248                (
2249                    "double_col",
2250                    Arc::new(Float64Array::from_iter_values(
2251                        (0..8).map(|x| (x % 2) as f64 * 10.1),
2252                    )) as _,
2253                    true,
2254                ),
2255                (
2256                    "date_string_col",
2257                    Arc::new(BinaryArray::from_iter_values([
2258                        [48, 51, 47, 48, 49, 47, 48, 57],
2259                        [48, 51, 47, 48, 49, 47, 48, 57],
2260                        [48, 52, 47, 48, 49, 47, 48, 57],
2261                        [48, 52, 47, 48, 49, 47, 48, 57],
2262                        [48, 50, 47, 48, 49, 47, 48, 57],
2263                        [48, 50, 47, 48, 49, 47, 48, 57],
2264                        [48, 49, 47, 48, 49, 47, 48, 57],
2265                        [48, 49, 47, 48, 49, 47, 48, 57],
2266                    ])) as _,
2267                    true,
2268                ),
2269                (
2270                    "string_col",
2271                    Arc::new(BinaryArray::from_iter_values((0..8).map(|x| [48 + x % 2]))) as _,
2272                    true,
2273                ),
2274                (
2275                    "timestamp_col",
2276                    Arc::new(
2277                        TimestampMicrosecondArray::from_iter_values([
2278                            1235865600000000, // 2009-03-01T00:00:00.000
2279                            1235865660000000, // 2009-03-01T00:01:00.000
2280                            1238544000000000, // 2009-04-01T00:00:00.000
2281                            1238544060000000, // 2009-04-01T00:01:00.000
2282                            1233446400000000, // 2009-02-01T00:00:00.000
2283                            1233446460000000, // 2009-02-01T00:01:00.000
2284                            1230768000000000, // 2009-01-01T00:00:00.000
2285                            1230768060000000, // 2009-01-01T00:01:00.000
2286                        ])
2287                        .with_timezone("+00:00"),
2288                    ) as _,
2289                    true,
2290                ),
2291            ])
2292            .unwrap();
2293            assert_eq!(batch, expected, "mismatch for file {file}");
2294        }
2295    }
2296
2297    #[test]
2298    fn test_alltypes_schema_promotion_bytes_to_string_only() {
2299        for file in files() {
2300            let file = arrow_test_data(file);
2301            let mut promotions: HashMap<&str, &str> = HashMap::new();
2302            promotions.insert("date_string_col", "string");
2303            promotions.insert("string_col", "string");
2304            let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2305            let batch = read_alltypes_with_reader_schema(&file, reader_schema);
2306            let expected = RecordBatch::try_from_iter_with_nullable([
2307                (
2308                    "id",
2309                    Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
2310                    true,
2311                ),
2312                (
2313                    "bool_col",
2314                    Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
2315                    true,
2316                ),
2317                (
2318                    "tinyint_col",
2319                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2320                    true,
2321                ),
2322                (
2323                    "smallint_col",
2324                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2325                    true,
2326                ),
2327                (
2328                    "int_col",
2329                    Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
2330                    true,
2331                ),
2332                (
2333                    "bigint_col",
2334                    Arc::new(Int64Array::from_iter_values((0..8).map(|x| (x % 2) * 10))) as _,
2335                    true,
2336                ),
2337                (
2338                    "float_col",
2339                    Arc::new(Float32Array::from_iter_values(
2340                        (0..8).map(|x| (x % 2) as f32 * 1.1),
2341                    )) as _,
2342                    true,
2343                ),
2344                (
2345                    "double_col",
2346                    Arc::new(Float64Array::from_iter_values(
2347                        (0..8).map(|x| (x % 2) as f64 * 10.1),
2348                    )) as _,
2349                    true,
2350                ),
2351                (
2352                    "date_string_col",
2353                    Arc::new(StringArray::from(vec![
2354                        "03/01/09", "03/01/09", "04/01/09", "04/01/09", "02/01/09", "02/01/09",
2355                        "01/01/09", "01/01/09",
2356                    ])) as _,
2357                    true,
2358                ),
2359                (
2360                    "string_col",
2361                    Arc::new(StringArray::from(
2362                        (0..8)
2363                            .map(|x| if x % 2 == 0 { "0" } else { "1" })
2364                            .collect::<Vec<_>>(),
2365                    )) as _,
2366                    true,
2367                ),
2368                (
2369                    "timestamp_col",
2370                    Arc::new(
2371                        TimestampMicrosecondArray::from_iter_values([
2372                            1235865600000000, // 2009-03-01T00:00:00.000
2373                            1235865660000000, // 2009-03-01T00:01:00.000
2374                            1238544000000000, // 2009-04-01T00:00:00.000
2375                            1238544060000000, // 2009-04-01T00:01:00.000
2376                            1233446400000000, // 2009-02-01T00:00:00.000
2377                            1233446460000000, // 2009-02-01T00:01:00.000
2378                            1230768000000000, // 2009-01-01T00:00:00.000
2379                            1230768060000000, // 2009-01-01T00:01:00.000
2380                        ])
2381                        .with_timezone("+00:00"),
2382                    ) as _,
2383                    true,
2384                ),
2385            ])
2386            .unwrap();
2387            assert_eq!(batch, expected, "mismatch for file {file}");
2388        }
2389    }
2390
2391    #[test]
2392    // TODO: avoid requiring snappy for this file
2393    #[cfg(feature = "snappy")]
2394    fn test_alltypes_illegal_promotion_bool_to_double_errors() {
2395        let file = arrow_test_data("avro/alltypes_plain.avro");
2396        let mut promotions: HashMap<&str, &str> = HashMap::new();
2397        promotions.insert("bool_col", "double"); // illegal
2398        let reader_schema = make_reader_schema_with_promotions(&file, &promotions);
2399        let file_handle = File::open(&file).unwrap();
2400        let result = ReaderBuilder::new()
2401            .with_reader_schema(reader_schema)
2402            .build(BufReader::new(file_handle));
2403        let err = result.expect_err("expected illegal promotion to error");
2404        let msg = err.to_string();
2405        assert!(
2406            msg.contains("Illegal promotion") || msg.contains("illegal promotion"),
2407            "unexpected error: {msg}"
2408        );
2409    }
2410
2411    #[test]
2412    fn test_simple_enum_with_reader_schema_mapping() {
2413        let file = arrow_test_data("avro/simple_enum.avro");
2414        let mut remap: HashMap<&str, Vec<&str>> = HashMap::new();
2415        remap.insert("f1", vec!["d", "c", "b", "a"]);
2416        remap.insert("f2", vec!["h", "g", "f", "e"]);
2417        remap.insert("f3", vec!["k", "i", "j"]);
2418        let reader_schema = make_reader_schema_with_enum_remap(&file, &remap);
2419        let actual = read_alltypes_with_reader_schema(&file, reader_schema);
2420        let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8));
2421        // f1
2422        let f1_keys = Int32Array::from(vec![3, 2, 1, 0]);
2423        let f1_vals = StringArray::from(vec!["d", "c", "b", "a"]);
2424        let f1 = DictionaryArray::<Int32Type>::try_new(f1_keys, Arc::new(f1_vals)).unwrap();
2425        let mut md_f1 = HashMap::new();
2426        md_f1.insert(
2427            AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
2428            r#"["d","c","b","a"]"#.to_string(),
2429        );
2430        // New named-type metadata
2431        md_f1.insert("avro.name".to_string(), "enum1".to_string());
2432        md_f1.insert("avro.namespace".to_string(), "ns1".to_string());
2433        let f1_field = Field::new("f1", dict_type.clone(), false).with_metadata(md_f1);
2434        // f2
2435        let f2_keys = Int32Array::from(vec![1, 0, 3, 2]);
2436        let f2_vals = StringArray::from(vec!["h", "g", "f", "e"]);
2437        let f2 = DictionaryArray::<Int32Type>::try_new(f2_keys, Arc::new(f2_vals)).unwrap();
2438        let mut md_f2 = HashMap::new();
2439        md_f2.insert(
2440            AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
2441            r#"["h","g","f","e"]"#.to_string(),
2442        );
2443        // New named-type metadata
2444        md_f2.insert("avro.name".to_string(), "enum2".to_string());
2445        md_f2.insert("avro.namespace".to_string(), "ns2".to_string());
2446        let f2_field = Field::new("f2", dict_type.clone(), false).with_metadata(md_f2);
2447        // f3
2448        let f3_keys = Int32Array::from(vec![Some(2), Some(0), None, Some(1)]);
2449        let f3_vals = StringArray::from(vec!["k", "i", "j"]);
2450        let f3 = DictionaryArray::<Int32Type>::try_new(f3_keys, Arc::new(f3_vals)).unwrap();
2451        let mut md_f3 = HashMap::new();
2452        md_f3.insert(
2453            AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
2454            r#"["k","i","j"]"#.to_string(),
2455        );
2456        // New named-type metadata
2457        md_f3.insert("avro.name".to_string(), "enum3".to_string());
2458        md_f3.insert("avro.namespace".to_string(), "ns1".to_string());
2459        let f3_field = Field::new("f3", dict_type.clone(), true).with_metadata(md_f3);
2460        let expected_schema = Arc::new(Schema::new(vec![f1_field, f2_field, f3_field]));
2461        let expected = RecordBatch::try_new(
2462            expected_schema,
2463            vec![Arc::new(f1) as ArrayRef, Arc::new(f2), Arc::new(f3)],
2464        )
2465        .unwrap();
2466        assert_eq!(actual, expected);
2467    }
2468
2469    #[test]
2470    fn test_schema_store_register_lookup() {
2471        let schema_int = make_record_schema(PrimitiveType::Int);
2472        let schema_long = make_record_schema(PrimitiveType::Long);
2473        let mut store = SchemaStore::new();
2474        let fp_int = store.register(schema_int.clone()).unwrap();
2475        let fp_long = store.register(schema_long.clone()).unwrap();
2476        assert_eq!(store.lookup(&fp_int).cloned(), Some(schema_int));
2477        assert_eq!(store.lookup(&fp_long).cloned(), Some(schema_long));
2478        assert_eq!(store.fingerprint_algorithm(), FingerprintAlgorithm::Rabin);
2479    }
2480
2481    #[test]
2482    fn test_unknown_fingerprint_is_error() {
2483        let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store();
2484        let unknown_fp = Fingerprint::Rabin(0xDEAD_BEEF_DEAD_BEEF);
2485        let prefix = make_prefix(unknown_fp);
2486        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2487        let err = decoder.decode(&prefix).expect_err("decode should error");
2488        let msg = err.to_string();
2489        assert!(
2490            msg.contains("Unknown fingerprint"),
2491            "unexpected message: {msg}"
2492        );
2493    }
2494
2495    #[test]
2496    fn test_handle_prefix_incomplete_magic() {
2497        let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store();
2498        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2499        let buf = &SINGLE_OBJECT_MAGIC[..1];
2500        let res = decoder.handle_prefix(buf).unwrap();
2501        assert_eq!(res, Some(0));
2502        assert!(decoder.pending_schema.is_none());
2503    }
2504
2505    #[test]
2506    fn test_handle_prefix_magic_mismatch() {
2507        let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store();
2508        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2509        let buf = [0xFFu8, 0x00u8, 0x01u8];
2510        let res = decoder.handle_prefix(&buf).unwrap();
2511        assert!(res.is_none());
2512    }
2513
2514    #[test]
2515    fn test_handle_prefix_incomplete_fingerprint() {
2516        let (store, fp_int, fp_long, _schema_int, schema_long) = make_two_schema_store();
2517        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2518        let long_bytes = match fp_long {
2519            Fingerprint::Rabin(v) => v.to_le_bytes(),
2520            Fingerprint::Id(id) => panic!("expected Rabin fingerprint, got ({id})"),
2521            Fingerprint::Id64(id) => panic!("expected Rabin fingerprint, got ({id})"),
2522            #[cfg(feature = "md5")]
2523            Fingerprint::MD5(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2524            #[cfg(feature = "sha256")]
2525            Fingerprint::SHA256(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2526        };
2527        let mut buf = Vec::from(SINGLE_OBJECT_MAGIC);
2528        buf.extend_from_slice(&long_bytes[..4]);
2529        let res = decoder.handle_prefix(&buf).unwrap();
2530        assert_eq!(res, Some(0));
2531        assert!(decoder.pending_schema.is_none());
2532    }
2533
2534    #[test]
2535    fn test_handle_prefix_valid_prefix_switches_schema() {
2536        let (store, fp_int, fp_long, _schema_int, schema_long) = make_two_schema_store();
2537        let mut decoder = make_decoder(&store, fp_int, &schema_long);
2538        let writer_schema_long = schema_long.schema().unwrap();
2539        let root_long = AvroFieldBuilder::new(&writer_schema_long).build().unwrap();
2540        let long_decoder = RecordDecoder::try_new_with_options(root_long.data_type()).unwrap();
2541        let _ = decoder.cache.insert(fp_long, long_decoder);
2542        let mut buf = Vec::from(SINGLE_OBJECT_MAGIC);
2543        match fp_long {
2544            Fingerprint::Rabin(v) => buf.extend_from_slice(&v.to_le_bytes()),
2545            Fingerprint::Id(id) => panic!("expected Rabin fingerprint, got ({id})"),
2546            Fingerprint::Id64(id) => panic!("expected Rabin fingerprint, got ({id})"),
2547            #[cfg(feature = "md5")]
2548            Fingerprint::MD5(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2549            #[cfg(feature = "sha256")]
2550            Fingerprint::SHA256(v) => panic!("expected Rabin fingerprint, got ({v:?})"),
2551        }
2552        let consumed = decoder.handle_prefix(&buf).unwrap().unwrap();
2553        assert_eq!(consumed, buf.len());
2554        assert!(decoder.pending_schema.is_some());
2555        assert_eq!(decoder.pending_schema.as_ref().unwrap().0, fp_long);
2556    }
2557
2558    #[test]
2559    fn test_decoder_projection_multiple_writer_schemas_no_reader_schema()
2560    -> Result<(), Box<dyn std::error::Error>> {
2561        // Two writer schemas with different shapes
2562        let writer_v1 = AvroSchema::new(
2563            r#"{"type":"record","name":"E","fields":[{"name":"a","type":"int"},{"name":"b","type":"string"}]}"#
2564                .to_string(),
2565        );
2566        let writer_v2 = AvroSchema::new(
2567            r#"{"type":"record","name":"E","fields":[{"name":"a","type":"long"},{"name":"b","type":"string"},{"name":"c","type":"int"}]}"#
2568                .to_string(),
2569        );
2570        let mut store = SchemaStore::new();
2571        let fp1 = store.register(writer_v1)?;
2572        let fp2 = store.register(writer_v2)?;
2573        let mut decoder = ReaderBuilder::new()
2574            .with_writer_schema_store(store)
2575            .with_active_fingerprint(fp1)
2576            .with_batch_size(8)
2577            .with_projection(vec![1])
2578            .build_decoder()?;
2579        // Message for v1: {a:1, b:"x"}
2580        let mut msg1 = make_prefix(fp1);
2581        msg1.extend_from_slice(&encode_zigzag(1)); // a = 1
2582        msg1.push((1u8) << 1);
2583        msg1.extend_from_slice(b"x");
2584        // Message for v2: {a:2, b:"y", c:7}
2585        let mut msg2 = make_prefix(fp2);
2586        msg2.extend_from_slice(&encode_zigzag(2)); // a = 2
2587        msg2.push((1u8) << 1);
2588        msg2.extend_from_slice(b"y");
2589        msg2.extend_from_slice(&encode_zigzag(7)); // c = 7
2590        decoder.decode(&msg1)?;
2591        let batch1 = decoder.flush()?.expect("batch1");
2592        assert_eq!(batch1.num_columns(), 1);
2593        assert_eq!(batch1.schema().field(0).name(), "b");
2594        let b1 = batch1.column(0).as_string::<i32>();
2595        assert_eq!(b1.value(0), "x");
2596        decoder.decode(&msg2)?;
2597        let batch2 = decoder.flush()?.expect("batch2");
2598        assert_eq!(batch2.num_columns(), 1);
2599        assert_eq!(batch2.schema().field(0).name(), "b");
2600        let b2 = batch2.column(0).as_string::<i32>();
2601        assert_eq!(b2.value(0), "y");
2602        Ok(())
2603    }
2604
2605    #[test]
2606    fn test_two_messages_same_schema() {
2607        let writer_schema = make_value_schema(PrimitiveType::Int);
2608        let reader_schema = writer_schema.clone();
2609        let mut store = SchemaStore::new();
2610        let fp = store.register(writer_schema).unwrap();
2611        let msg1 = make_message(fp, 42);
2612        let msg2 = make_message(fp, 11);
2613        let input = [msg1.clone(), msg2.clone()].concat();
2614        let mut decoder = ReaderBuilder::new()
2615            .with_batch_size(8)
2616            .with_reader_schema(reader_schema.clone())
2617            .with_writer_schema_store(store)
2618            .with_active_fingerprint(fp)
2619            .build_decoder()
2620            .unwrap();
2621        let _ = decoder.decode(&input).unwrap();
2622        let batch = decoder.flush().unwrap().expect("batch");
2623        assert_eq!(batch.num_rows(), 2);
2624        let col = batch
2625            .column(0)
2626            .as_any()
2627            .downcast_ref::<Int32Array>()
2628            .unwrap();
2629        assert_eq!(col.value(0), 42);
2630        assert_eq!(col.value(1), 11);
2631    }
2632
2633    #[test]
2634    fn test_two_messages_schema_switch() {
2635        let w_int = make_value_schema(PrimitiveType::Int);
2636        let w_long = make_value_schema(PrimitiveType::Long);
2637        let mut store = SchemaStore::new();
2638        let fp_int = store.register(w_int).unwrap();
2639        let fp_long = store.register(w_long).unwrap();
2640        let msg_int = make_message(fp_int, 1);
2641        let msg_long = make_message(fp_long, 123456789_i64);
2642        let mut decoder = ReaderBuilder::new()
2643            .with_batch_size(8)
2644            .with_writer_schema_store(store)
2645            .with_active_fingerprint(fp_int)
2646            .build_decoder()
2647            .unwrap();
2648        let _ = decoder.decode(&msg_int).unwrap();
2649        let batch1 = decoder.flush().unwrap().expect("batch1");
2650        assert_eq!(batch1.num_rows(), 1);
2651        assert_eq!(
2652            batch1
2653                .column(0)
2654                .as_any()
2655                .downcast_ref::<Int32Array>()
2656                .unwrap()
2657                .value(0),
2658            1
2659        );
2660        let _ = decoder.decode(&msg_long).unwrap();
2661        let batch2 = decoder.flush().unwrap().expect("batch2");
2662        assert_eq!(batch2.num_rows(), 1);
2663        assert_eq!(
2664            batch2
2665                .column(0)
2666                .as_any()
2667                .downcast_ref::<Int64Array>()
2668                .unwrap()
2669                .value(0),
2670            123456789_i64
2671        );
2672    }
2673
2674    #[test]
2675    fn test_two_messages_same_schema_id() {
2676        let writer_schema = make_value_schema(PrimitiveType::Int);
2677        let reader_schema = writer_schema.clone();
2678        let id = 100u32;
2679        // Set up store with None fingerprint algorithm and register schema by id
2680        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
2681        let _ = store
2682            .set(Fingerprint::Id(id), writer_schema.clone())
2683            .expect("set id schema");
2684        let msg1 = make_message_id(id, 21);
2685        let msg2 = make_message_id(id, 22);
2686        let input = [msg1.clone(), msg2.clone()].concat();
2687        let mut decoder = ReaderBuilder::new()
2688            .with_batch_size(8)
2689            .with_reader_schema(reader_schema)
2690            .with_writer_schema_store(store)
2691            .with_active_fingerprint(Fingerprint::Id(id))
2692            .build_decoder()
2693            .unwrap();
2694        let _ = decoder.decode(&input).unwrap();
2695        let batch = decoder.flush().unwrap().expect("batch");
2696        assert_eq!(batch.num_rows(), 2);
2697        let col = batch
2698            .column(0)
2699            .as_any()
2700            .downcast_ref::<Int32Array>()
2701            .unwrap();
2702        assert_eq!(col.value(0), 21);
2703        assert_eq!(col.value(1), 22);
2704    }
2705
2706    #[test]
2707    fn test_unknown_id_fingerprint_is_error() {
2708        let writer_schema = make_value_schema(PrimitiveType::Int);
2709        let id_known = 7u32;
2710        let id_unknown = 9u32;
2711        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
2712        let _ = store
2713            .set(Fingerprint::Id(id_known), writer_schema.clone())
2714            .expect("set id schema");
2715        let mut decoder = ReaderBuilder::new()
2716            .with_batch_size(8)
2717            .with_reader_schema(writer_schema)
2718            .with_writer_schema_store(store)
2719            .with_active_fingerprint(Fingerprint::Id(id_known))
2720            .build_decoder()
2721            .unwrap();
2722        let prefix = make_id_prefix(id_unknown, 0);
2723        let err = decoder.decode(&prefix).expect_err("decode should error");
2724        let msg = err.to_string();
2725        assert!(
2726            msg.contains("Unknown fingerprint"),
2727            "unexpected message: {msg}"
2728        );
2729    }
2730
2731    #[test]
2732    fn test_handle_prefix_id_incomplete_magic() {
2733        let writer_schema = make_value_schema(PrimitiveType::Int);
2734        let id = 5u32;
2735        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id);
2736        let _ = store
2737            .set(Fingerprint::Id(id), writer_schema.clone())
2738            .expect("set id schema");
2739        let mut decoder = ReaderBuilder::new()
2740            .with_batch_size(8)
2741            .with_reader_schema(writer_schema)
2742            .with_writer_schema_store(store)
2743            .with_active_fingerprint(Fingerprint::Id(id))
2744            .build_decoder()
2745            .unwrap();
2746        let buf = &CONFLUENT_MAGIC[..0]; // empty incomplete magic
2747        let res = decoder.handle_prefix(buf).unwrap();
2748        assert_eq!(res, Some(0));
2749        assert!(decoder.pending_schema.is_none());
2750    }
2751
2752    #[test]
2753    fn test_two_messages_same_schema_id64() {
2754        let writer_schema = make_value_schema(PrimitiveType::Int);
2755        let reader_schema = writer_schema.clone();
2756        let id = 100u64;
2757        // Set up store with None fingerprint algorithm and register schema by id
2758        let mut store = SchemaStore::new_with_type(FingerprintAlgorithm::Id64);
2759        let _ = store
2760            .set(Fingerprint::Id64(id), writer_schema.clone())
2761            .expect("set id schema");
2762        let msg1 = make_message_id64(id, 21);
2763        let msg2 = make_message_id64(id, 22);
2764        let input = [msg1.clone(), msg2.clone()].concat();
2765        let mut decoder = ReaderBuilder::new()
2766            .with_batch_size(8)
2767            .with_reader_schema(reader_schema)
2768            .with_writer_schema_store(store)
2769            .with_active_fingerprint(Fingerprint::Id64(id))
2770            .build_decoder()
2771            .unwrap();
2772        let _ = decoder.decode(&input).unwrap();
2773        let batch = decoder.flush().unwrap().expect("batch");
2774        assert_eq!(batch.num_rows(), 2);
2775        let col = batch
2776            .column(0)
2777            .as_any()
2778            .downcast_ref::<Int32Array>()
2779            .unwrap();
2780        assert_eq!(col.value(0), 21);
2781        assert_eq!(col.value(1), 22);
2782    }
2783
2784    #[test]
2785    fn test_decode_stream_with_schema() {
2786        struct TestCase<'a> {
2787            name: &'a str,
2788            schema: &'a str,
2789            expected_error: Option<&'a str>,
2790        }
2791        let tests = vec![
2792            TestCase {
2793                name: "success",
2794                schema: r#"{"type":"record","name":"test","fields":[{"name":"f2","type":"string"}]}"#,
2795                expected_error: None,
2796            },
2797            TestCase {
2798                name: "valid schema invalid data",
2799                schema: r#"{"type":"record","name":"test","fields":[{"name":"f2","type":"long"}]}"#,
2800                expected_error: Some("did not consume all bytes"),
2801            },
2802        ];
2803        for test in tests {
2804            let avro_schema = AvroSchema::new(test.schema.to_string());
2805            let mut store = SchemaStore::new();
2806            let fp = store.register(avro_schema.clone()).unwrap();
2807            let prefix = make_prefix(fp);
2808            let record_val = "some_string";
2809            let mut body = prefix;
2810            body.push((record_val.len() as u8) << 1);
2811            body.extend_from_slice(record_val.as_bytes());
2812            let decoder_res = ReaderBuilder::new()
2813                .with_batch_size(1)
2814                .with_writer_schema_store(store)
2815                .with_active_fingerprint(fp)
2816                .build_decoder();
2817            let decoder = match decoder_res {
2818                Ok(d) => d,
2819                Err(e) => {
2820                    if let Some(expected) = test.expected_error {
2821                        assert!(
2822                            e.to_string().contains(expected),
2823                            "Test '{}' failed at build – expected '{expected}', got '{e}'",
2824                            test.name
2825                        );
2826                        continue;
2827                    } else {
2828                        panic!("Test '{}' failed during build: {e}", test.name);
2829                    }
2830                }
2831            };
2832            let stream = Box::pin(stream::once(async { Bytes::from(body) }));
2833            let decoded_stream = decode_stream(decoder, stream);
2834            let batches_result: Result<Vec<RecordBatch>, ArrowError> =
2835                block_on(decoded_stream.try_collect());
2836            match (batches_result, test.expected_error) {
2837                (Ok(batches), None) => {
2838                    let batch =
2839                        arrow::compute::concat_batches(&batches[0].schema(), &batches).unwrap();
2840                    let expected_field = Field::new("f2", DataType::Utf8, false);
2841                    let expected_schema = Arc::new(Schema::new(vec![expected_field]));
2842                    let expected_array = Arc::new(StringArray::from(vec![record_val]));
2843                    let expected_batch =
2844                        RecordBatch::try_new(expected_schema, vec![expected_array]).unwrap();
2845                    assert_eq!(batch, expected_batch, "Test '{}'", test.name);
2846                }
2847                (Err(e), Some(expected)) => {
2848                    assert!(
2849                        e.to_string().contains(expected),
2850                        "Test '{}' – expected error containing '{expected}', got '{e}'",
2851                        test.name
2852                    );
2853                }
2854                (Ok(_), Some(expected)) => {
2855                    panic!(
2856                        "Test '{}' expected failure ('{expected}') but succeeded",
2857                        test.name
2858                    );
2859                }
2860                (Err(e), None) => {
2861                    panic!("Test '{}' unexpectedly failed with '{e}'", test.name);
2862                }
2863            }
2864        }
2865    }
2866
2867    #[test]
2868    fn test_utf8view_support() {
2869        struct TestHelper;
2870        impl TestHelper {
2871            fn with_utf8view(field: &Field) -> Field {
2872                match field.data_type() {
2873                    DataType::Utf8 => {
2874                        Field::new(field.name(), DataType::Utf8View, field.is_nullable())
2875                            .with_metadata(field.metadata().clone())
2876                    }
2877                    _ => field.clone(),
2878                }
2879            }
2880        }
2881
2882        let field = TestHelper::with_utf8view(&Field::new("str_field", DataType::Utf8, false));
2883
2884        assert_eq!(field.data_type(), &DataType::Utf8View);
2885
2886        let array = StringViewArray::from(vec!["test1", "test2"]);
2887        let batch =
2888            RecordBatch::try_from_iter(vec![("str_field", Arc::new(array) as ArrayRef)]).unwrap();
2889
2890        assert!(batch.column(0).as_any().is::<StringViewArray>());
2891    }
2892
2893    fn make_reader_schema_with_default_fields(
2894        path: &str,
2895        default_fields: Vec<Value>,
2896    ) -> AvroSchema {
2897        let mut root = load_writer_schema_json(path);
2898        assert_eq!(root["type"], "record", "writer schema must be a record");
2899        root.as_object_mut()
2900            .expect("schema is a JSON object")
2901            .insert("fields".to_string(), Value::Array(default_fields));
2902        AvroSchema::new(root.to_string())
2903    }
2904
2905    #[test]
2906    fn test_schema_resolution_defaults_all_supported_types() {
2907        let path = "test/data/skippable_types.avro";
2908        let duration_default = "\u{0000}".repeat(12);
2909        let reader_schema = make_reader_schema_with_default_fields(
2910            path,
2911            vec![
2912                serde_json::json!({"name":"d_bool","type":"boolean","default":true}),
2913                serde_json::json!({"name":"d_int","type":"int","default":42}),
2914                serde_json::json!({"name":"d_long","type":"long","default":12345}),
2915                serde_json::json!({"name":"d_float","type":"float","default":1.5}),
2916                serde_json::json!({"name":"d_double","type":"double","default":2.25}),
2917                serde_json::json!({"name":"d_bytes","type":"bytes","default":"XYZ"}),
2918                serde_json::json!({"name":"d_string","type":"string","default":"hello"}),
2919                serde_json::json!({"name":"d_date","type":{"type":"int","logicalType":"date"},"default":0}),
2920                serde_json::json!({"name":"d_time_ms","type":{"type":"int","logicalType":"time-millis"},"default":1000}),
2921                serde_json::json!({"name":"d_time_us","type":{"type":"long","logicalType":"time-micros"},"default":2000}),
2922                serde_json::json!({"name":"d_ts_ms","type":{"type":"long","logicalType":"local-timestamp-millis"},"default":0}),
2923                serde_json::json!({"name":"d_ts_us","type":{"type":"long","logicalType":"local-timestamp-micros"},"default":0}),
2924                serde_json::json!({"name":"d_decimal","type":{"type":"bytes","logicalType":"decimal","precision":10,"scale":2},"default":""}),
2925                serde_json::json!({"name":"d_fixed","type":{"type":"fixed","name":"F4","size":4},"default":"ABCD"}),
2926                serde_json::json!({"name":"d_enum","type":{"type":"enum","name":"E","symbols":["A","B","C"]},"default":"A"}),
2927                serde_json::json!({"name":"d_duration","type":{"type":"fixed","name":"Dur","size":12,"logicalType":"duration"},"default":duration_default}),
2928                serde_json::json!({"name":"d_uuid","type":{"type":"string","logicalType":"uuid"},"default":"00000000-0000-0000-0000-000000000000"}),
2929                serde_json::json!({"name":"d_array","type":{"type":"array","items":"int"},"default":[1,2,3]}),
2930                serde_json::json!({"name":"d_map","type":{"type":"map","values":"long"},"default":{"a":1,"b":2}}),
2931                serde_json::json!({"name":"d_record","type":{
2932              "type":"record","name":"DefaultRec","fields":[
2933                  {"name":"x","type":"int"},
2934                  {"name":"y","type":["null","string"],"default":null}
2935              ]
2936        },"default":{"x":7}}),
2937                serde_json::json!({"name":"d_nullable_null","type":["null","int"],"default":null}),
2938                serde_json::json!({"name":"d_nullable_value","type":["int","null"],"default":123}),
2939            ],
2940        );
2941        let actual = read_alltypes_with_reader_schema(path, reader_schema);
2942        let num_rows = actual.num_rows();
2943        assert!(num_rows > 0, "skippable_types.avro should contain rows");
2944        assert_eq!(
2945            actual.num_columns(),
2946            22,
2947            "expected exactly our defaulted fields"
2948        );
2949        let mut arrays: Vec<Arc<dyn Array>> = Vec::with_capacity(22);
2950        arrays.push(Arc::new(BooleanArray::from_iter(std::iter::repeat_n(
2951            Some(true),
2952            num_rows,
2953        ))));
2954        arrays.push(Arc::new(Int32Array::from_iter_values(std::iter::repeat_n(
2955            42, num_rows,
2956        ))));
2957        arrays.push(Arc::new(Int64Array::from_iter_values(std::iter::repeat_n(
2958            12345, num_rows,
2959        ))));
2960        arrays.push(Arc::new(Float32Array::from_iter_values(
2961            std::iter::repeat_n(1.5f32, num_rows),
2962        )));
2963        arrays.push(Arc::new(Float64Array::from_iter_values(
2964            std::iter::repeat_n(2.25f64, num_rows),
2965        )));
2966        arrays.push(Arc::new(BinaryArray::from_iter_values(
2967            std::iter::repeat_n(b"XYZ".as_ref(), num_rows),
2968        )));
2969        arrays.push(Arc::new(StringArray::from_iter_values(
2970            std::iter::repeat_n("hello", num_rows),
2971        )));
2972        arrays.push(Arc::new(Date32Array::from_iter_values(
2973            std::iter::repeat_n(0, num_rows),
2974        )));
2975        arrays.push(Arc::new(Time32MillisecondArray::from_iter_values(
2976            std::iter::repeat_n(1_000, num_rows),
2977        )));
2978        arrays.push(Arc::new(Time64MicrosecondArray::from_iter_values(
2979            std::iter::repeat_n(2_000i64, num_rows),
2980        )));
2981        arrays.push(Arc::new(TimestampMillisecondArray::from_iter_values(
2982            std::iter::repeat_n(0i64, num_rows),
2983        )));
2984        arrays.push(Arc::new(TimestampMicrosecondArray::from_iter_values(
2985            std::iter::repeat_n(0i64, num_rows),
2986        )));
2987        #[cfg(feature = "small_decimals")]
2988        let decimal = Decimal64Array::from_iter_values(std::iter::repeat_n(0i64, num_rows))
2989            .with_precision_and_scale(10, 2)
2990            .unwrap();
2991        #[cfg(not(feature = "small_decimals"))]
2992        let decimal = Decimal128Array::from_iter_values(std::iter::repeat_n(0i128, num_rows))
2993            .with_precision_and_scale(10, 2)
2994            .unwrap();
2995        arrays.push(Arc::new(decimal));
2996        let fixed_iter = std::iter::repeat_n(Some(*b"ABCD"), num_rows);
2997        arrays.push(Arc::new(
2998            FixedSizeBinaryArray::try_from_sparse_iter_with_size(fixed_iter, 4).unwrap(),
2999        ));
3000        let enum_keys = Int32Array::from_iter_values(std::iter::repeat_n(0, num_rows));
3001        let enum_values = StringArray::from_iter_values(["A", "B", "C"]);
3002        let enum_arr =
3003            DictionaryArray::<Int32Type>::try_new(enum_keys, Arc::new(enum_values)).unwrap();
3004        arrays.push(Arc::new(enum_arr));
3005        let duration_values = std::iter::repeat_n(
3006            Some(IntervalMonthDayNanoType::make_value(0, 0, 0)),
3007            num_rows,
3008        );
3009        let duration_arr: IntervalMonthDayNanoArray = duration_values.collect();
3010        arrays.push(Arc::new(duration_arr));
3011        let uuid_bytes = [0u8; 16];
3012        let uuid_iter = std::iter::repeat_n(Some(uuid_bytes), num_rows);
3013        arrays.push(Arc::new(
3014            FixedSizeBinaryArray::try_from_sparse_iter_with_size(uuid_iter, 16).unwrap(),
3015        ));
3016        let item_field = Arc::new(Field::new(
3017            Field::LIST_FIELD_DEFAULT_NAME,
3018            DataType::Int32,
3019            false,
3020        ));
3021        let mut list_builder = ListBuilder::new(Int32Builder::new()).with_field(item_field);
3022        for _ in 0..num_rows {
3023            list_builder.values().append_value(1);
3024            list_builder.values().append_value(2);
3025            list_builder.values().append_value(3);
3026            list_builder.append(true);
3027        }
3028        arrays.push(Arc::new(list_builder.finish()));
3029        let values_field = Arc::new(Field::new("value", DataType::Int64, false));
3030        let mut map_builder = MapBuilder::new(
3031            Some(builder::MapFieldNames {
3032                entry: "entries".to_string(),
3033                key: "key".to_string(),
3034                value: "value".to_string(),
3035            }),
3036            StringBuilder::new(),
3037            Int64Builder::new(),
3038        )
3039        .with_values_field(values_field);
3040        for _ in 0..num_rows {
3041            let (keys, vals) = map_builder.entries();
3042            keys.append_value("a");
3043            vals.append_value(1);
3044            keys.append_value("b");
3045            vals.append_value(2);
3046            map_builder.append(true).unwrap();
3047        }
3048        arrays.push(Arc::new(map_builder.finish()));
3049        let rec_fields: Fields = Fields::from(vec![
3050            Field::new("x", DataType::Int32, false),
3051            Field::new("y", DataType::Utf8, true),
3052        ]);
3053        let mut sb = StructBuilder::new(
3054            rec_fields.clone(),
3055            vec![
3056                Box::new(Int32Builder::new()),
3057                Box::new(StringBuilder::new()),
3058            ],
3059        );
3060        for _ in 0..num_rows {
3061            sb.field_builder::<Int32Builder>(0).unwrap().append_value(7);
3062            sb.field_builder::<StringBuilder>(1).unwrap().append_null();
3063            sb.append(true);
3064        }
3065        arrays.push(Arc::new(sb.finish()));
3066        arrays.push(Arc::new(Int32Array::from_iter(std::iter::repeat_n(
3067            None::<i32>,
3068            num_rows,
3069        ))));
3070        arrays.push(Arc::new(Int32Array::from_iter_values(std::iter::repeat_n(
3071            123, num_rows,
3072        ))));
3073        let expected = RecordBatch::try_new(actual.schema(), arrays).unwrap();
3074        assert_eq!(
3075            actual, expected,
3076            "defaults should materialize correctly for all fields"
3077        );
3078    }
3079
3080    #[test]
3081    fn test_schema_resolution_default_enum_invalid_symbol_errors() {
3082        let path = "test/data/skippable_types.avro";
3083        let bad_schema = make_reader_schema_with_default_fields(
3084            path,
3085            vec![serde_json::json!({
3086                "name":"bad_enum",
3087                "type":{"type":"enum","name":"E","symbols":["A","B","C"]},
3088                "default":"Z"
3089            })],
3090        );
3091        let file = File::open(path).unwrap();
3092        let res = ReaderBuilder::new()
3093            .with_reader_schema(bad_schema)
3094            .build(BufReader::new(file));
3095        let err = res.expect_err("expected enum default validation to fail");
3096        let msg = err.to_string();
3097        let lower_msg = msg.to_lowercase();
3098        assert!(
3099            lower_msg.contains("enum")
3100                && (lower_msg.contains("symbol") || lower_msg.contains("default")),
3101            "unexpected error: {msg}"
3102        );
3103    }
3104
3105    #[test]
3106    fn test_schema_resolution_default_fixed_size_mismatch_errors() {
3107        let path = "test/data/skippable_types.avro";
3108        let bad_schema = make_reader_schema_with_default_fields(
3109            path,
3110            vec![serde_json::json!({
3111                "name":"bad_fixed",
3112                "type":{"type":"fixed","name":"F","size":4},
3113                "default":"ABC"
3114            })],
3115        );
3116        let file = File::open(path).unwrap();
3117        let res = ReaderBuilder::new()
3118            .with_reader_schema(bad_schema)
3119            .build(BufReader::new(file));
3120        let err = res.expect_err("expected fixed default validation to fail");
3121        let msg = err.to_string();
3122        let lower_msg = msg.to_lowercase();
3123        assert!(
3124            lower_msg.contains("fixed")
3125                && (lower_msg.contains("size")
3126                    || lower_msg.contains("length")
3127                    || lower_msg.contains("does not match")),
3128            "unexpected error: {msg}"
3129        );
3130    }
3131
3132    #[test]
3133    // TODO: avoid requiring snappy for this file
3134    #[cfg(feature = "snappy")]
3135    fn test_alltypes_skip_writer_fields_keep_double_only() {
3136        let file = arrow_test_data("avro/alltypes_plain.avro");
3137        let reader_schema =
3138            make_reader_schema_with_selected_fields_in_order(&file, &["double_col"]);
3139        let batch = read_alltypes_with_reader_schema(&file, reader_schema);
3140        let expected = RecordBatch::try_from_iter_with_nullable([(
3141            "double_col",
3142            Arc::new(Float64Array::from_iter_values(
3143                (0..8).map(|x| (x % 2) as f64 * 10.1),
3144            )) as _,
3145            true,
3146        )])
3147        .unwrap();
3148        assert_eq!(batch, expected);
3149    }
3150
3151    #[test]
3152    // TODO: avoid requiring snappy for this file
3153    #[cfg(feature = "snappy")]
3154    fn test_alltypes_skip_writer_fields_reorder_and_skip_many() {
3155        let file = arrow_test_data("avro/alltypes_plain.avro");
3156        let reader_schema =
3157            make_reader_schema_with_selected_fields_in_order(&file, &["timestamp_col", "id"]);
3158        let batch = read_alltypes_with_reader_schema(&file, reader_schema);
3159        let expected = RecordBatch::try_from_iter_with_nullable([
3160            (
3161                "timestamp_col",
3162                Arc::new(
3163                    TimestampMicrosecondArray::from_iter_values([
3164                        1235865600000000, // 2009-03-01T00:00:00.000
3165                        1235865660000000, // 2009-03-01T00:01:00.000
3166                        1238544000000000, // 2009-04-01T00:00:00.000
3167                        1238544060000000, // 2009-04-01T00:01:00.000
3168                        1233446400000000, // 2009-02-01T00:00:00.000
3169                        1233446460000000, // 2009-02-01T00:01:00.000
3170                        1230768000000000, // 2009-01-01T00:00:00.000
3171                        1230768060000000, // 2009-01-01T00:01:00.000
3172                    ])
3173                    .with_timezone("+00:00"),
3174                ) as _,
3175                true,
3176            ),
3177            (
3178                "id",
3179                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
3180                true,
3181            ),
3182        ])
3183        .unwrap();
3184        assert_eq!(batch, expected);
3185    }
3186
3187    #[test]
3188    fn test_skippable_types_project_each_field_individually() {
3189        let path = "test/data/skippable_types.avro";
3190        let full = read_file(path, 1024, false);
3191        let schema_full = full.schema();
3192        let num_rows = full.num_rows();
3193        let writer_json = load_writer_schema_json(path);
3194        assert_eq!(
3195            writer_json["type"], "record",
3196            "writer schema must be a record"
3197        );
3198        let fields_json = writer_json
3199            .get("fields")
3200            .and_then(|f| f.as_array())
3201            .expect("record has fields");
3202        assert_eq!(
3203            schema_full.fields().len(),
3204            fields_json.len(),
3205            "full read column count vs writer fields"
3206        );
3207        fn rebuild_list_array_with_element(
3208            col: &ArrayRef,
3209            new_elem: Arc<Field>,
3210            is_large: bool,
3211        ) -> ArrayRef {
3212            if is_large {
3213                let list = col
3214                    .as_any()
3215                    .downcast_ref::<LargeListArray>()
3216                    .expect("expected LargeListArray");
3217                let offsets = list.offsets().clone();
3218                let values = list.values().clone();
3219                let validity = list.nulls().cloned();
3220                Arc::new(LargeListArray::try_new(new_elem, offsets, values, validity).unwrap())
3221            } else {
3222                let list = col
3223                    .as_any()
3224                    .downcast_ref::<ListArray>()
3225                    .expect("expected ListArray");
3226                let offsets = list.offsets().clone();
3227                let values = list.values().clone();
3228                let validity = list.nulls().cloned();
3229                Arc::new(ListArray::try_new(new_elem, offsets, values, validity).unwrap())
3230            }
3231        }
3232        for (idx, f) in fields_json.iter().enumerate() {
3233            let name = f
3234                .get("name")
3235                .and_then(|n| n.as_str())
3236                .unwrap_or_else(|| panic!("field at index {idx} has no name"));
3237            let reader_schema = make_reader_schema_with_selected_fields_in_order(path, &[name]);
3238            let projected = read_alltypes_with_reader_schema(path, reader_schema);
3239            assert_eq!(
3240                projected.num_columns(),
3241                1,
3242                "projected batch should contain exactly the selected column '{name}'"
3243            );
3244            assert_eq!(
3245                projected.num_rows(),
3246                num_rows,
3247                "row count mismatch for projected column '{name}'"
3248            );
3249            let col_full = full.column(idx).clone();
3250            let full_field = schema_full.field(idx).as_ref().clone();
3251            let proj_field_ref = projected.schema().field(0).clone();
3252            let proj_field = proj_field_ref.as_ref();
3253            let top_meta = proj_field.metadata().clone();
3254            let (expected_field_ref, expected_col): (Arc<Field>, ArrayRef) =
3255                match (full_field.data_type(), proj_field.data_type()) {
3256                    (&DataType::List(_), DataType::List(proj_elem)) => {
3257                        let new_col =
3258                            rebuild_list_array_with_element(&col_full, proj_elem.clone(), false);
3259                        let nf = Field::new(
3260                            full_field.name().clone(),
3261                            proj_field.data_type().clone(),
3262                            full_field.is_nullable(),
3263                        )
3264                        .with_metadata(top_meta);
3265                        (Arc::new(nf), new_col)
3266                    }
3267                    (&DataType::LargeList(_), DataType::LargeList(proj_elem)) => {
3268                        let new_col =
3269                            rebuild_list_array_with_element(&col_full, proj_elem.clone(), true);
3270                        let nf = Field::new(
3271                            full_field.name().clone(),
3272                            proj_field.data_type().clone(),
3273                            full_field.is_nullable(),
3274                        )
3275                        .with_metadata(top_meta);
3276                        (Arc::new(nf), new_col)
3277                    }
3278                    _ => {
3279                        let nf = full_field.with_metadata(top_meta);
3280                        (Arc::new(nf), col_full)
3281                    }
3282                };
3283
3284            let expected = RecordBatch::try_new(
3285                Arc::new(Schema::new(vec![expected_field_ref])),
3286                vec![expected_col],
3287            )
3288            .unwrap();
3289            assert_eq!(
3290                projected, expected,
3291                "projected column '{name}' mismatch vs full read column"
3292            );
3293        }
3294    }
3295
3296    #[test]
3297    fn test_union_fields_avro_nullable_and_general_unions() {
3298        let path = "test/data/union_fields.avro";
3299        let batch = read_file(path, 1024, false);
3300        let schema = batch.schema();
3301        let idx = schema.index_of("nullable_int_nullfirst").unwrap();
3302        let a = batch.column(idx).as_primitive::<Int32Type>();
3303        assert_eq!(a.len(), 4);
3304        assert!(a.is_null(0));
3305        assert_eq!(a.value(1), 42);
3306        assert!(a.is_null(2));
3307        assert_eq!(a.value(3), 0);
3308        let idx = schema.index_of("nullable_string_nullsecond").unwrap();
3309        let s = batch
3310            .column(idx)
3311            .as_any()
3312            .downcast_ref::<StringArray>()
3313            .expect("nullable_string_nullsecond should be Utf8");
3314        assert_eq!(s.len(), 4);
3315        assert_eq!(s.value(0), "s1");
3316        assert!(s.is_null(1));
3317        assert_eq!(s.value(2), "s3");
3318        assert!(s.is_valid(3)); // empty string, not null
3319        assert_eq!(s.value(3), "");
3320        let idx = schema.index_of("union_prim").unwrap();
3321        let u = batch
3322            .column(idx)
3323            .as_any()
3324            .downcast_ref::<UnionArray>()
3325            .expect("union_prim should be Union");
3326        let fields = match u.data_type() {
3327            DataType::Union(fields, mode) => {
3328                assert!(matches!(mode, UnionMode::Dense), "expect dense unions");
3329                fields
3330            }
3331            other => panic!("expected Union, got {other:?}"),
3332        };
3333        let tid_by_name = |name: &str| -> i8 {
3334            for (tid, f) in fields.iter() {
3335                if f.name() == name {
3336                    return tid;
3337                }
3338            }
3339            panic!("union child '{name}' not found");
3340        };
3341        let expected_type_ids = vec![
3342            tid_by_name("long"),
3343            tid_by_name("int"),
3344            tid_by_name("float"),
3345            tid_by_name("double"),
3346        ];
3347        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3348        assert_eq!(
3349            type_ids, expected_type_ids,
3350            "branch selection for union_prim rows"
3351        );
3352        let longs = u
3353            .child(tid_by_name("long"))
3354            .as_any()
3355            .downcast_ref::<Int64Array>()
3356            .unwrap();
3357        assert_eq!(longs.len(), 1);
3358        let ints = u
3359            .child(tid_by_name("int"))
3360            .as_any()
3361            .downcast_ref::<Int32Array>()
3362            .unwrap();
3363        assert_eq!(ints.len(), 1);
3364        let floats = u
3365            .child(tid_by_name("float"))
3366            .as_any()
3367            .downcast_ref::<Float32Array>()
3368            .unwrap();
3369        assert_eq!(floats.len(), 1);
3370        let doubles = u
3371            .child(tid_by_name("double"))
3372            .as_any()
3373            .downcast_ref::<Float64Array>()
3374            .unwrap();
3375        assert_eq!(doubles.len(), 1);
3376        let idx = schema.index_of("union_bytes_vs_string").unwrap();
3377        let u = batch
3378            .column(idx)
3379            .as_any()
3380            .downcast_ref::<UnionArray>()
3381            .expect("union_bytes_vs_string should be Union");
3382        let fields = match u.data_type() {
3383            DataType::Union(fields, _) => fields,
3384            other => panic!("expected Union, got {other:?}"),
3385        };
3386        let tid_by_name = |name: &str| -> i8 {
3387            for (tid, f) in fields.iter() {
3388                if f.name() == name {
3389                    return tid;
3390                }
3391            }
3392            panic!("union child '{name}' not found");
3393        };
3394        let tid_bytes = tid_by_name("bytes");
3395        let tid_string = tid_by_name("string");
3396        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3397        assert_eq!(
3398            type_ids,
3399            vec![tid_bytes, tid_string, tid_string, tid_bytes],
3400            "branch selection for bytes/string union"
3401        );
3402        let s_child = u
3403            .child(tid_string)
3404            .as_any()
3405            .downcast_ref::<StringArray>()
3406            .unwrap();
3407        assert_eq!(s_child.len(), 2);
3408        assert_eq!(s_child.value(0), "hello");
3409        assert_eq!(s_child.value(1), "world");
3410        let b_child = u
3411            .child(tid_bytes)
3412            .as_any()
3413            .downcast_ref::<BinaryArray>()
3414            .unwrap();
3415        assert_eq!(b_child.len(), 2);
3416        assert_eq!(b_child.value(0), &[0x00, 0xFF, 0x7F]);
3417        assert_eq!(b_child.value(1), b""); // previously: &[]
3418        let idx = schema.index_of("union_enum_records_array_map").unwrap();
3419        let u = batch
3420            .column(idx)
3421            .as_any()
3422            .downcast_ref::<UnionArray>()
3423            .expect("union_enum_records_array_map should be Union");
3424        let fields = match u.data_type() {
3425            DataType::Union(fields, _) => fields,
3426            other => panic!("expected Union, got {other:?}"),
3427        };
3428        let mut tid_enum: Option<i8> = None;
3429        let mut tid_rec_a: Option<i8> = None;
3430        let mut tid_rec_b: Option<i8> = None;
3431        let mut tid_array: Option<i8> = None;
3432        for (tid, f) in fields.iter() {
3433            match f.data_type() {
3434                DataType::Dictionary(_, _) => tid_enum = Some(tid),
3435                DataType::Struct(childs) => {
3436                    if childs.len() == 2 && childs[0].name() == "a" && childs[1].name() == "b" {
3437                        tid_rec_a = Some(tid);
3438                    } else if childs.len() == 2
3439                        && childs[0].name() == "x"
3440                        && childs[1].name() == "y"
3441                    {
3442                        tid_rec_b = Some(tid);
3443                    }
3444                }
3445                DataType::List(_) => tid_array = Some(tid),
3446                _ => {}
3447            }
3448        }
3449        let (tid_enum, tid_rec_a, tid_rec_b, tid_array) = (
3450            tid_enum.expect("enum child"),
3451            tid_rec_a.expect("RecA child"),
3452            tid_rec_b.expect("RecB child"),
3453            tid_array.expect("array<long> child"),
3454        );
3455        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3456        assert_eq!(
3457            type_ids,
3458            vec![tid_enum, tid_rec_a, tid_rec_b, tid_array],
3459            "branch selection for complex union"
3460        );
3461        let dict = u
3462            .child(tid_enum)
3463            .as_any()
3464            .downcast_ref::<DictionaryArray<Int32Type>>()
3465            .unwrap();
3466        assert_eq!(dict.len(), 1);
3467        assert!(dict.is_valid(0));
3468        let rec_a = u
3469            .child(tid_rec_a)
3470            .as_any()
3471            .downcast_ref::<StructArray>()
3472            .unwrap();
3473        assert_eq!(rec_a.len(), 1);
3474        let a_val = rec_a
3475            .column_by_name("a")
3476            .unwrap()
3477            .as_any()
3478            .downcast_ref::<Int32Array>()
3479            .unwrap();
3480        assert_eq!(a_val.value(0), 7);
3481        let b_val = rec_a
3482            .column_by_name("b")
3483            .unwrap()
3484            .as_any()
3485            .downcast_ref::<StringArray>()
3486            .unwrap();
3487        assert_eq!(b_val.value(0), "x");
3488        // RecB row: {"x": 123456789, "y": b"\xFF\x00"}
3489        let rec_b = u
3490            .child(tid_rec_b)
3491            .as_any()
3492            .downcast_ref::<StructArray>()
3493            .unwrap();
3494        let x_val = rec_b
3495            .column_by_name("x")
3496            .unwrap()
3497            .as_any()
3498            .downcast_ref::<Int64Array>()
3499            .unwrap();
3500        assert_eq!(x_val.value(0), 123_456_789_i64);
3501        let y_val = rec_b
3502            .column_by_name("y")
3503            .unwrap()
3504            .as_any()
3505            .downcast_ref::<BinaryArray>()
3506            .unwrap();
3507        assert_eq!(y_val.value(0), &[0xFF, 0x00]);
3508        let arr = u
3509            .child(tid_array)
3510            .as_any()
3511            .downcast_ref::<ListArray>()
3512            .unwrap();
3513        assert_eq!(arr.len(), 1);
3514        let first_values = arr.value(0);
3515        let longs = first_values.as_any().downcast_ref::<Int64Array>().unwrap();
3516        assert_eq!(longs.len(), 3);
3517        assert_eq!(longs.value(0), 1);
3518        assert_eq!(longs.value(1), 2);
3519        assert_eq!(longs.value(2), 3);
3520        let idx = schema.index_of("union_date_or_fixed4").unwrap();
3521        let u = batch
3522            .column(idx)
3523            .as_any()
3524            .downcast_ref::<UnionArray>()
3525            .expect("union_date_or_fixed4 should be Union");
3526        let fields = match u.data_type() {
3527            DataType::Union(fields, _) => fields,
3528            other => panic!("expected Union, got {other:?}"),
3529        };
3530        let mut tid_date: Option<i8> = None;
3531        let mut tid_fixed: Option<i8> = None;
3532        for (tid, f) in fields.iter() {
3533            match f.data_type() {
3534                DataType::Date32 => tid_date = Some(tid),
3535                DataType::FixedSizeBinary(4) => tid_fixed = Some(tid),
3536                _ => {}
3537            }
3538        }
3539        let (tid_date, tid_fixed) = (tid_date.expect("date"), tid_fixed.expect("fixed(4)"));
3540        let type_ids: Vec<i8> = u.type_ids().iter().copied().collect();
3541        assert_eq!(
3542            type_ids,
3543            vec![tid_date, tid_fixed, tid_date, tid_fixed],
3544            "branch selection for date/fixed4 union"
3545        );
3546        let dates = u
3547            .child(tid_date)
3548            .as_any()
3549            .downcast_ref::<Date32Array>()
3550            .unwrap();
3551        assert_eq!(dates.len(), 2);
3552        assert_eq!(dates.value(0), 19_000); // ~2022‑01‑15
3553        assert_eq!(dates.value(1), 0); // epoch
3554        let fixed = u
3555            .child(tid_fixed)
3556            .as_any()
3557            .downcast_ref::<FixedSizeBinaryArray>()
3558            .unwrap();
3559        assert_eq!(fixed.len(), 2);
3560        assert_eq!(fixed.value(0), b"ABCD");
3561        assert_eq!(fixed.value(1), &[0x00, 0x11, 0x22, 0x33]);
3562    }
3563
3564    #[test]
3565    fn test_union_schema_resolution_all_type_combinations() {
3566        let path = "test/data/union_fields.avro";
3567        let baseline = read_file(path, 1024, false);
3568        let baseline_schema = baseline.schema();
3569        let mut root = load_writer_schema_json(path);
3570        assert_eq!(root["type"], "record", "writer schema must be a record");
3571        let fields = root
3572            .get_mut("fields")
3573            .and_then(|f| f.as_array_mut())
3574            .expect("record has fields");
3575        fn is_named_type(obj: &Value, ty: &str, nm: &str) -> bool {
3576            obj.get("type").and_then(|v| v.as_str()) == Some(ty)
3577                && obj.get("name").and_then(|v| v.as_str()) == Some(nm)
3578        }
3579        fn is_logical(obj: &Value, prim: &str, lt: &str) -> bool {
3580            obj.get("type").and_then(|v| v.as_str()) == Some(prim)
3581                && obj.get("logicalType").and_then(|v| v.as_str()) == Some(lt)
3582        }
3583        fn find_first(arr: &[Value], pred: impl Fn(&Value) -> bool) -> Option<Value> {
3584            arr.iter().find(|v| pred(v)).cloned()
3585        }
3586        fn prim(s: &str) -> Value {
3587            Value::String(s.to_string())
3588        }
3589        for f in fields.iter_mut() {
3590            let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
3591                continue;
3592            };
3593            match name {
3594                // Flip null ordering – should not affect values
3595                "nullable_int_nullfirst" => {
3596                    f["type"] = json!(["int", "null"]);
3597                }
3598                "nullable_string_nullsecond" => {
3599                    f["type"] = json!(["null", "string"]);
3600                }
3601                "union_prim" => {
3602                    let orig = f["type"].as_array().unwrap().clone();
3603                    let long = prim("long");
3604                    let double = prim("double");
3605                    let string = prim("string");
3606                    let bytes = prim("bytes");
3607                    let boolean = prim("boolean");
3608                    assert!(orig.contains(&long));
3609                    assert!(orig.contains(&double));
3610                    assert!(orig.contains(&string));
3611                    assert!(orig.contains(&bytes));
3612                    assert!(orig.contains(&boolean));
3613                    f["type"] = json!([long, double, string, bytes, boolean]);
3614                }
3615                "union_bytes_vs_string" => {
3616                    f["type"] = json!(["string", "bytes"]);
3617                }
3618                "union_fixed_dur_decfix" => {
3619                    let orig = f["type"].as_array().unwrap().clone();
3620                    let fx8 = find_first(&orig, |o| is_named_type(o, "fixed", "Fx8")).unwrap();
3621                    let dur12 = find_first(&orig, |o| is_named_type(o, "fixed", "Dur12")).unwrap();
3622                    let decfix16 =
3623                        find_first(&orig, |o| is_named_type(o, "fixed", "DecFix16")).unwrap();
3624                    f["type"] = json!([decfix16, dur12, fx8]);
3625                }
3626                "union_enum_records_array_map" => {
3627                    let orig = f["type"].as_array().unwrap().clone();
3628                    let enum_color = find_first(&orig, |o| {
3629                        o.get("type").and_then(|v| v.as_str()) == Some("enum")
3630                    })
3631                    .unwrap();
3632                    let rec_a = find_first(&orig, |o| is_named_type(o, "record", "RecA")).unwrap();
3633                    let rec_b = find_first(&orig, |o| is_named_type(o, "record", "RecB")).unwrap();
3634                    let arr = find_first(&orig, |o| {
3635                        o.get("type").and_then(|v| v.as_str()) == Some("array")
3636                    })
3637                    .unwrap();
3638                    let map = find_first(&orig, |o| {
3639                        o.get("type").and_then(|v| v.as_str()) == Some("map")
3640                    })
3641                    .unwrap();
3642                    f["type"] = json!([arr, map, rec_b, rec_a, enum_color]);
3643                }
3644                "union_date_or_fixed4" => {
3645                    let orig = f["type"].as_array().unwrap().clone();
3646                    let date = find_first(&orig, |o| is_logical(o, "int", "date")).unwrap();
3647                    let fx4 = find_first(&orig, |o| is_named_type(o, "fixed", "Fx4")).unwrap();
3648                    f["type"] = json!([fx4, date]);
3649                }
3650                "union_time_millis_or_enum" => {
3651                    let orig = f["type"].as_array().unwrap().clone();
3652                    let time_ms =
3653                        find_first(&orig, |o| is_logical(o, "int", "time-millis")).unwrap();
3654                    let en = find_first(&orig, |o| {
3655                        o.get("type").and_then(|v| v.as_str()) == Some("enum")
3656                    })
3657                    .unwrap();
3658                    f["type"] = json!([en, time_ms]);
3659                }
3660                "union_time_micros_or_string" => {
3661                    let orig = f["type"].as_array().unwrap().clone();
3662                    let time_us =
3663                        find_first(&orig, |o| is_logical(o, "long", "time-micros")).unwrap();
3664                    f["type"] = json!(["string", time_us]);
3665                }
3666                "union_ts_millis_utc_or_array" => {
3667                    let orig = f["type"].as_array().unwrap().clone();
3668                    let ts_ms =
3669                        find_first(&orig, |o| is_logical(o, "long", "timestamp-millis")).unwrap();
3670                    let arr = find_first(&orig, |o| {
3671                        o.get("type").and_then(|v| v.as_str()) == Some("array")
3672                    })
3673                    .unwrap();
3674                    f["type"] = json!([arr, ts_ms]);
3675                }
3676                "union_ts_micros_local_or_bytes" => {
3677                    let orig = f["type"].as_array().unwrap().clone();
3678                    let lts_us =
3679                        find_first(&orig, |o| is_logical(o, "long", "local-timestamp-micros"))
3680                            .unwrap();
3681                    f["type"] = json!(["bytes", lts_us]);
3682                }
3683                "union_uuid_or_fixed10" => {
3684                    let orig = f["type"].as_array().unwrap().clone();
3685                    let uuid = find_first(&orig, |o| is_logical(o, "string", "uuid")).unwrap();
3686                    let fx10 = find_first(&orig, |o| is_named_type(o, "fixed", "Fx10")).unwrap();
3687                    f["type"] = json!([fx10, uuid]);
3688                }
3689                "union_dec_bytes_or_dec_fixed" => {
3690                    let orig = f["type"].as_array().unwrap().clone();
3691                    let dec_bytes = find_first(&orig, |o| {
3692                        o.get("type").and_then(|v| v.as_str()) == Some("bytes")
3693                            && o.get("logicalType").and_then(|v| v.as_str()) == Some("decimal")
3694                    })
3695                    .unwrap();
3696                    let dec_fix = find_first(&orig, |o| {
3697                        is_named_type(o, "fixed", "DecFix20")
3698                            && o.get("logicalType").and_then(|v| v.as_str()) == Some("decimal")
3699                    })
3700                    .unwrap();
3701                    f["type"] = json!([dec_fix, dec_bytes]);
3702                }
3703                "union_null_bytes_string" => {
3704                    f["type"] = json!(["bytes", "string", "null"]);
3705                }
3706                "array_of_union" => {
3707                    let obj = f
3708                        .get_mut("type")
3709                        .expect("array type")
3710                        .as_object_mut()
3711                        .unwrap();
3712                    obj.insert("items".to_string(), json!(["string", "long"]));
3713                }
3714                "map_of_union" => {
3715                    let obj = f
3716                        .get_mut("type")
3717                        .expect("map type")
3718                        .as_object_mut()
3719                        .unwrap();
3720                    obj.insert("values".to_string(), json!(["double", "null"]));
3721                }
3722                "record_with_union_field" => {
3723                    let rec = f
3724                        .get_mut("type")
3725                        .expect("record type")
3726                        .as_object_mut()
3727                        .unwrap();
3728                    let rec_fields = rec.get_mut("fields").unwrap().as_array_mut().unwrap();
3729                    let mut found = false;
3730                    for rf in rec_fields.iter_mut() {
3731                        if rf.get("name").and_then(|v| v.as_str()) == Some("u") {
3732                            rf["type"] = json!(["string", "long"]); // rely on int→long promotion
3733                            found = true;
3734                            break;
3735                        }
3736                    }
3737                    assert!(found, "field 'u' expected in HasUnion");
3738                }
3739                "union_ts_micros_utc_or_map" => {
3740                    let orig = f["type"].as_array().unwrap().clone();
3741                    let ts_us =
3742                        find_first(&orig, |o| is_logical(o, "long", "timestamp-micros")).unwrap();
3743                    let map = find_first(&orig, |o| {
3744                        o.get("type").and_then(|v| v.as_str()) == Some("map")
3745                    })
3746                    .unwrap();
3747                    f["type"] = json!([map, ts_us]);
3748                }
3749                "union_ts_millis_local_or_string" => {
3750                    let orig = f["type"].as_array().unwrap().clone();
3751                    let lts_ms =
3752                        find_first(&orig, |o| is_logical(o, "long", "local-timestamp-millis"))
3753                            .unwrap();
3754                    f["type"] = json!(["string", lts_ms]);
3755                }
3756                "union_bool_or_string" => {
3757                    f["type"] = json!(["string", "boolean"]);
3758                }
3759                _ => {}
3760            }
3761        }
3762        let reader_schema = AvroSchema::new(root.to_string());
3763        let resolved = read_alltypes_with_reader_schema(path, reader_schema);
3764
3765        fn branch_token(dt: &DataType) -> String {
3766            match dt {
3767                DataType::Null => "null".into(),
3768                DataType::Boolean => "boolean".into(),
3769                DataType::Int32 => "int".into(),
3770                DataType::Int64 => "long".into(),
3771                DataType::Float32 => "float".into(),
3772                DataType::Float64 => "double".into(),
3773                DataType::Binary => "bytes".into(),
3774                DataType::Utf8 => "string".into(),
3775                DataType::Date32 => "date".into(),
3776                DataType::Time32(arrow_schema::TimeUnit::Millisecond) => "time-millis".into(),
3777                DataType::Time64(arrow_schema::TimeUnit::Microsecond) => "time-micros".into(),
3778                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => if tz.is_some() {
3779                    "timestamp-millis"
3780                } else {
3781                    "local-timestamp-millis"
3782                }
3783                .into(),
3784                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => if tz.is_some() {
3785                    "timestamp-micros"
3786                } else {
3787                    "local-timestamp-micros"
3788                }
3789                .into(),
3790                DataType::Interval(IntervalUnit::MonthDayNano) => "duration".into(),
3791                DataType::FixedSizeBinary(n) => format!("fixed{n}"),
3792                DataType::Dictionary(_, _) => "enum".into(),
3793                DataType::Decimal128(p, s) => format!("decimal({p},{s})"),
3794                DataType::Decimal256(p, s) => format!("decimal({p},{s})"),
3795                #[cfg(feature = "small_decimals")]
3796                DataType::Decimal64(p, s) => format!("decimal({p},{s})"),
3797                DataType::Struct(fields) => {
3798                    if fields.len() == 2 && fields[0].name() == "a" && fields[1].name() == "b" {
3799                        "record:RecA".into()
3800                    } else if fields.len() == 2
3801                        && fields[0].name() == "x"
3802                        && fields[1].name() == "y"
3803                    {
3804                        "record:RecB".into()
3805                    } else {
3806                        "record".into()
3807                    }
3808                }
3809                DataType::List(_) => "array".into(),
3810                DataType::Map(_, _) => "map".into(),
3811                other => format!("{other:?}"),
3812            }
3813        }
3814
3815        fn union_tokens(u: &UnionArray) -> (Vec<i8>, HashMap<i8, String>) {
3816            let fields = match u.data_type() {
3817                DataType::Union(fields, _) => fields,
3818                other => panic!("expected Union, got {other:?}"),
3819            };
3820            let mut dict: HashMap<i8, String> = HashMap::with_capacity(fields.len());
3821            for (tid, f) in fields.iter() {
3822                dict.insert(tid, branch_token(f.data_type()));
3823            }
3824            let ids: Vec<i8> = u.type_ids().iter().copied().collect();
3825            (ids, dict)
3826        }
3827
3828        fn expected_token(field_name: &str, writer_token: &str) -> String {
3829            match field_name {
3830                "union_prim" => match writer_token {
3831                    "int" => "long".into(),
3832                    "float" => "double".into(),
3833                    other => other.into(),
3834                },
3835                "record_with_union_field.u" => match writer_token {
3836                    "int" => "long".into(),
3837                    other => other.into(),
3838                },
3839                _ => writer_token.into(),
3840            }
3841        }
3842
3843        fn get_union<'a>(
3844            rb: &'a RecordBatch,
3845            schema: arrow_schema::SchemaRef,
3846            fname: &str,
3847        ) -> &'a UnionArray {
3848            let idx = schema.index_of(fname).unwrap();
3849            rb.column(idx)
3850                .as_any()
3851                .downcast_ref::<UnionArray>()
3852                .unwrap_or_else(|| panic!("{fname} should be a Union"))
3853        }
3854
3855        fn assert_union_equivalent(field_name: &str, u_writer: &UnionArray, u_reader: &UnionArray) {
3856            let (ids_w, dict_w) = union_tokens(u_writer);
3857            let (ids_r, dict_r) = union_tokens(u_reader);
3858            assert_eq!(
3859                ids_w.len(),
3860                ids_r.len(),
3861                "{field_name}: row count mismatch between baseline and resolved"
3862            );
3863            for (i, (id_w, id_r)) in ids_w.iter().zip(ids_r.iter()).enumerate() {
3864                let w_tok = dict_w.get(id_w).unwrap();
3865                let want = expected_token(field_name, w_tok);
3866                let got = dict_r.get(id_r).unwrap();
3867                assert_eq!(
3868                    got, &want,
3869                    "{field_name}: row {i} resolved to wrong union branch (writer={w_tok}, expected={want}, got={got})"
3870                );
3871            }
3872        }
3873
3874        for (fname, dt) in [
3875            ("nullable_int_nullfirst", DataType::Int32),
3876            ("nullable_string_nullsecond", DataType::Utf8),
3877        ] {
3878            let idx_b = baseline_schema.index_of(fname).unwrap();
3879            let idx_r = resolved.schema().index_of(fname).unwrap();
3880            let col_b = baseline.column(idx_b);
3881            let col_r = resolved.column(idx_r);
3882            assert_eq!(
3883                col_b.data_type(),
3884                &dt,
3885                "baseline {fname} should decode as non-union with nullability"
3886            );
3887            assert_eq!(
3888                col_b.as_ref(),
3889                col_r.as_ref(),
3890                "{fname}: values must be identical regardless of null-branch order"
3891            );
3892        }
3893        let union_fields = [
3894            "union_prim",
3895            "union_bytes_vs_string",
3896            "union_fixed_dur_decfix",
3897            "union_enum_records_array_map",
3898            "union_date_or_fixed4",
3899            "union_time_millis_or_enum",
3900            "union_time_micros_or_string",
3901            "union_ts_millis_utc_or_array",
3902            "union_ts_micros_local_or_bytes",
3903            "union_uuid_or_fixed10",
3904            "union_dec_bytes_or_dec_fixed",
3905            "union_null_bytes_string",
3906            "union_ts_micros_utc_or_map",
3907            "union_ts_millis_local_or_string",
3908            "union_bool_or_string",
3909        ];
3910        for fname in union_fields {
3911            let u_b = get_union(&baseline, baseline_schema.clone(), fname);
3912            let u_r = get_union(&resolved, resolved.schema(), fname);
3913            assert_union_equivalent(fname, u_b, u_r);
3914        }
3915        {
3916            let fname = "array_of_union";
3917            let idx_b = baseline_schema.index_of(fname).unwrap();
3918            let idx_r = resolved.schema().index_of(fname).unwrap();
3919            let arr_b = baseline
3920                .column(idx_b)
3921                .as_any()
3922                .downcast_ref::<ListArray>()
3923                .expect("array_of_union should be a List");
3924            let arr_r = resolved
3925                .column(idx_r)
3926                .as_any()
3927                .downcast_ref::<ListArray>()
3928                .expect("array_of_union should be a List");
3929            assert_eq!(
3930                arr_b.value_offsets(),
3931                arr_r.value_offsets(),
3932                "{fname}: list offsets changed after resolution"
3933            );
3934            let u_b = arr_b
3935                .values()
3936                .as_any()
3937                .downcast_ref::<UnionArray>()
3938                .expect("array items should be Union");
3939            let u_r = arr_r
3940                .values()
3941                .as_any()
3942                .downcast_ref::<UnionArray>()
3943                .expect("array items should be Union");
3944            let (ids_b, dict_b) = union_tokens(u_b);
3945            let (ids_r, dict_r) = union_tokens(u_r);
3946            assert_eq!(ids_b.len(), ids_r.len(), "{fname}: values length mismatch");
3947            for (i, (id_b, id_r)) in ids_b.iter().zip(ids_r.iter()).enumerate() {
3948                let w_tok = dict_b.get(id_b).unwrap();
3949                let got = dict_r.get(id_r).unwrap();
3950                assert_eq!(
3951                    got, w_tok,
3952                    "{fname}: value {i} resolved to wrong branch (writer={w_tok}, got={got})"
3953                );
3954            }
3955        }
3956        {
3957            let fname = "map_of_union";
3958            let idx_b = baseline_schema.index_of(fname).unwrap();
3959            let idx_r = resolved.schema().index_of(fname).unwrap();
3960            let map_b = baseline
3961                .column(idx_b)
3962                .as_any()
3963                .downcast_ref::<MapArray>()
3964                .expect("map_of_union should be a Map");
3965            let map_r = resolved
3966                .column(idx_r)
3967                .as_any()
3968                .downcast_ref::<MapArray>()
3969                .expect("map_of_union should be a Map");
3970            assert_eq!(
3971                map_b.value_offsets(),
3972                map_r.value_offsets(),
3973                "{fname}: map value offsets changed after resolution"
3974            );
3975            let ent_b = map_b.entries();
3976            let ent_r = map_r.entries();
3977            let val_b_any = ent_b.column(1).as_ref();
3978            let val_r_any = ent_r.column(1).as_ref();
3979            let b_union = val_b_any.as_any().downcast_ref::<UnionArray>();
3980            let r_union = val_r_any.as_any().downcast_ref::<UnionArray>();
3981            if let (Some(u_b), Some(u_r)) = (b_union, r_union) {
3982                assert_union_equivalent(fname, u_b, u_r);
3983            } else {
3984                assert_eq!(
3985                    val_b_any.data_type(),
3986                    val_r_any.data_type(),
3987                    "{fname}: value data types differ after resolution"
3988                );
3989                assert_eq!(
3990                    val_b_any, val_r_any,
3991                    "{fname}: value arrays differ after resolution (nullable value column case)"
3992                );
3993                let value_nullable = |m: &MapArray| -> bool {
3994                    match m.data_type() {
3995                        DataType::Map(entries_field, _sorted) => match entries_field.data_type() {
3996                            DataType::Struct(fields) => {
3997                                assert_eq!(fields.len(), 2, "entries struct must have 2 fields");
3998                                assert_eq!(fields[0].name(), "key");
3999                                assert_eq!(fields[1].name(), "value");
4000                                fields[1].is_nullable()
4001                            }
4002                            other => panic!("Map entries field must be Struct, got {other:?}"),
4003                        },
4004                        other => panic!("expected Map data type, got {other:?}"),
4005                    }
4006                };
4007                assert!(
4008                    value_nullable(map_b),
4009                    "{fname}: baseline Map value field should be nullable per Arrow spec"
4010                );
4011                assert!(
4012                    value_nullable(map_r),
4013                    "{fname}: resolved Map value field should be nullable per Arrow spec"
4014                );
4015            }
4016        }
4017        {
4018            let fname = "record_with_union_field";
4019            let idx_b = baseline_schema.index_of(fname).unwrap();
4020            let idx_r = resolved.schema().index_of(fname).unwrap();
4021            let rec_b = baseline
4022                .column(idx_b)
4023                .as_any()
4024                .downcast_ref::<StructArray>()
4025                .expect("record_with_union_field should be a Struct");
4026            let rec_r = resolved
4027                .column(idx_r)
4028                .as_any()
4029                .downcast_ref::<StructArray>()
4030                .expect("record_with_union_field should be a Struct");
4031            let u_b = rec_b
4032                .column_by_name("u")
4033                .unwrap()
4034                .as_any()
4035                .downcast_ref::<UnionArray>()
4036                .expect("field 'u' should be Union (baseline)");
4037            let u_r = rec_r
4038                .column_by_name("u")
4039                .unwrap()
4040                .as_any()
4041                .downcast_ref::<UnionArray>()
4042                .expect("field 'u' should be Union (resolved)");
4043            assert_union_equivalent("record_with_union_field.u", u_b, u_r);
4044        }
4045    }
4046
4047    #[test]
4048    fn test_union_fields_end_to_end_expected_arrays() {
4049        fn tid_by_name(fields: &UnionFields, want: &str) -> i8 {
4050            for (tid, f) in fields.iter() {
4051                if f.name() == want {
4052                    return tid;
4053                }
4054            }
4055            panic!("union child '{want}' not found")
4056        }
4057
4058        fn tid_by_dt(fields: &UnionFields, pred: impl Fn(&DataType) -> bool) -> i8 {
4059            for (tid, f) in fields.iter() {
4060                if pred(f.data_type()) {
4061                    return tid;
4062                }
4063            }
4064            panic!("no union child matches predicate");
4065        }
4066
4067        fn uuid16_from_str(s: &str) -> [u8; 16] {
4068            fn hex(b: u8) -> u8 {
4069                match b {
4070                    b'0'..=b'9' => b - b'0',
4071                    b'a'..=b'f' => b - b'a' + 10,
4072                    b'A'..=b'F' => b - b'A' + 10,
4073                    _ => panic!("invalid hex"),
4074                }
4075            }
4076            let mut out = [0u8; 16];
4077            let bytes = s.as_bytes();
4078            let (mut i, mut j) = (0, 0);
4079            while i < bytes.len() {
4080                if bytes[i] == b'-' {
4081                    i += 1;
4082                    continue;
4083                }
4084                let hi = hex(bytes[i]);
4085                let lo = hex(bytes[i + 1]);
4086                out[j] = (hi << 4) | lo;
4087                j += 1;
4088                i += 2;
4089            }
4090            assert_eq!(j, 16, "uuid must decode to 16 bytes");
4091            out
4092        }
4093
4094        fn empty_child_for(dt: &DataType) -> Arc<dyn Array> {
4095            match dt {
4096                DataType::Null => Arc::new(NullArray::new(0)),
4097                DataType::Boolean => Arc::new(BooleanArray::from(Vec::<bool>::new())),
4098                DataType::Int32 => Arc::new(Int32Array::from(Vec::<i32>::new())),
4099                DataType::Int64 => Arc::new(Int64Array::from(Vec::<i64>::new())),
4100                DataType::Float32 => Arc::new(arrow_array::Float32Array::from(Vec::<f32>::new())),
4101                DataType::Float64 => Arc::new(arrow_array::Float64Array::from(Vec::<f64>::new())),
4102                DataType::Binary => Arc::new(BinaryArray::from(Vec::<&[u8]>::new())),
4103                DataType::Utf8 => Arc::new(StringArray::from(Vec::<&str>::new())),
4104                DataType::Date32 => Arc::new(arrow_array::Date32Array::from(Vec::<i32>::new())),
4105                DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
4106                    Arc::new(Time32MillisecondArray::from(Vec::<i32>::new()))
4107                }
4108                DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
4109                    Arc::new(Time64MicrosecondArray::from(Vec::<i64>::new()))
4110                }
4111                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
4112                    let a = TimestampMillisecondArray::from(Vec::<i64>::new());
4113                    Arc::new(if let Some(tz) = tz {
4114                        a.with_timezone(tz.clone())
4115                    } else {
4116                        a
4117                    })
4118                }
4119                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
4120                    let a = TimestampMicrosecondArray::from(Vec::<i64>::new());
4121                    Arc::new(if let Some(tz) = tz {
4122                        a.with_timezone(tz.clone())
4123                    } else {
4124                        a
4125                    })
4126                }
4127                DataType::Interval(IntervalUnit::MonthDayNano) => {
4128                    Arc::new(arrow_array::IntervalMonthDayNanoArray::from(Vec::<
4129                        IntervalMonthDayNano,
4130                    >::new(
4131                    )))
4132                }
4133                DataType::FixedSizeBinary(n) => Arc::new(FixedSizeBinaryArray::new_null(*n, 0)),
4134                DataType::Dictionary(k, v) => {
4135                    assert_eq!(**k, DataType::Int32, "expect int32 keys for enums");
4136                    let keys = Int32Array::from(Vec::<i32>::new());
4137                    let values = match v.as_ref() {
4138                        DataType::Utf8 => {
4139                            Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
4140                        }
4141                        other => panic!("unexpected dictionary value type {other:?}"),
4142                    };
4143                    Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
4144                }
4145                DataType::List(field) => {
4146                    let values: ArrayRef = match field.data_type() {
4147                        DataType::Int32 => {
4148                            Arc::new(Int32Array::from(Vec::<i32>::new())) as ArrayRef
4149                        }
4150                        DataType::Int64 => {
4151                            Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
4152                        }
4153                        DataType::Utf8 => {
4154                            Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
4155                        }
4156                        DataType::Union(_, _) => {
4157                            let (uf, _) = if let DataType::Union(f, m) = field.data_type() {
4158                                (f.clone(), m)
4159                            } else {
4160                                unreachable!()
4161                            };
4162                            let children: Vec<ArrayRef> = uf
4163                                .iter()
4164                                .map(|(_, f)| empty_child_for(f.data_type()))
4165                                .collect();
4166                            Arc::new(
4167                                UnionArray::try_new(
4168                                    uf.clone(),
4169                                    ScalarBuffer::<i8>::from(Vec::<i8>::new()),
4170                                    Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
4171                                    children,
4172                                )
4173                                .unwrap(),
4174                            ) as ArrayRef
4175                        }
4176                        other => panic!("unsupported list item type: {other:?}"),
4177                    };
4178                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
4179                    Arc::new(ListArray::try_new(field.clone(), offsets, values, None).unwrap())
4180                }
4181                DataType::Map(entry_field, ordered) => {
4182                    let DataType::Struct(childs) = entry_field.data_type() else {
4183                        panic!("map entries must be struct")
4184                    };
4185                    let key_field = &childs[0];
4186                    let val_field = &childs[1];
4187                    assert_eq!(key_field.data_type(), &DataType::Utf8);
4188                    let keys = StringArray::from(Vec::<&str>::new());
4189                    let vals: ArrayRef = match val_field.data_type() {
4190                        DataType::Float64 => {
4191                            Arc::new(arrow_array::Float64Array::from(Vec::<f64>::new())) as ArrayRef
4192                        }
4193                        DataType::Int64 => {
4194                            Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
4195                        }
4196                        DataType::Utf8 => {
4197                            Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
4198                        }
4199                        DataType::Union(uf, _) => {
4200                            let ch: Vec<ArrayRef> = uf
4201                                .iter()
4202                                .map(|(_, f)| empty_child_for(f.data_type()))
4203                                .collect();
4204                            Arc::new(
4205                                UnionArray::try_new(
4206                                    uf.clone(),
4207                                    ScalarBuffer::<i8>::from(Vec::<i8>::new()),
4208                                    Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
4209                                    ch,
4210                                )
4211                                .unwrap(),
4212                            ) as ArrayRef
4213                        }
4214                        other => panic!("unsupported map value type: {other:?}"),
4215                    };
4216                    let entries = StructArray::new(
4217                        Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
4218                        vec![Arc::new(keys) as ArrayRef, vals],
4219                        None,
4220                    );
4221                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
4222                    Arc::new(MapArray::new(
4223                        entry_field.clone(),
4224                        offsets,
4225                        entries,
4226                        None,
4227                        *ordered,
4228                    ))
4229                }
4230                other => panic!("empty_child_for: unhandled type {other:?}"),
4231            }
4232        }
4233
4234        fn mk_dense_union(
4235            fields: &UnionFields,
4236            type_ids: Vec<i8>,
4237            offsets: Vec<i32>,
4238            provide: impl Fn(&Field) -> Option<ArrayRef>,
4239        ) -> ArrayRef {
4240            let children: Vec<ArrayRef> = fields
4241                .iter()
4242                .map(|(_, f)| provide(f).unwrap_or_else(|| empty_child_for(f.data_type())))
4243                .collect();
4244
4245            Arc::new(
4246                UnionArray::try_new(
4247                    fields.clone(),
4248                    ScalarBuffer::<i8>::from(type_ids),
4249                    Some(ScalarBuffer::<i32>::from(offsets)),
4250                    children,
4251                )
4252                .unwrap(),
4253            ) as ArrayRef
4254        }
4255
4256        // Dates / times / timestamps from the Avro content block:
4257        let date_a: i32 = 19_000;
4258        let time_ms_a: i32 = 13 * 3_600_000 + 45 * 60_000 + 30_000 + 123;
4259        let time_us_b: i64 = 23 * 3_600_000_000 + 59 * 60_000_000 + 59 * 1_000_000 + 999_999;
4260        let ts_ms_2024_01_01: i64 = 1_704_067_200_000;
4261        let ts_us_2024_01_01: i64 = ts_ms_2024_01_01 * 1000;
4262        // Fixed / bytes-like values:
4263        let fx8_a: [u8; 8] = *b"ABCDEFGH";
4264        let fx4_abcd: [u8; 4] = *b"ABCD";
4265        let fx4_misc: [u8; 4] = [0x00, 0x11, 0x22, 0x33];
4266        let fx10_ascii: [u8; 10] = *b"0123456789";
4267        let fx10_aa: [u8; 10] = [0xAA; 10];
4268        // Duration logical values as MonthDayNano:
4269        let dur_a = IntervalMonthDayNanoType::make_value(1, 2, 3_000_000_000);
4270        let dur_b = IntervalMonthDayNanoType::make_value(12, 31, 999_000_000);
4271        // UUID logical values (stored as 16-byte FixedSizeBinary in Arrow):
4272        let uuid1 = uuid16_from_str("fe7bc30b-4ce8-4c5e-b67c-2234a2d38e66");
4273        let uuid2 = uuid16_from_str("0826cc06-d2e3-4599-b4ad-af5fa6905cdb");
4274        // Decimals from Avro content:
4275        let dec_b_scale2_pos: i128 = 123_456; // "1234.56" bytes-decimal -> (precision=10, scale=2)
4276        let dec_fix16_neg: i128 = -101; // "-1.01" fixed(16) decimal(10,2)
4277        let dec_fix20_s4: i128 = 1_234_567_891_234; // "123456789.1234" fixed(20) decimal(20,4)
4278        let dec_fix20_s4_neg: i128 = -123; // "-0.0123" fixed(20) decimal(20,4)
4279        let path = "test/data/union_fields.avro";
4280        let actual = read_file(path, 1024, false);
4281        let schema = actual.schema();
4282        // Helper to fetch union metadata for a column
4283        let get_union = |name: &str| -> (UnionFields, UnionMode) {
4284            let idx = schema.index_of(name).unwrap();
4285            match schema.field(idx).data_type() {
4286                DataType::Union(f, m) => (f.clone(), *m),
4287                other => panic!("{name} should be a Union, got {other:?}"),
4288            }
4289        };
4290        let mut expected_cols: Vec<ArrayRef> = Vec::with_capacity(schema.fields().len());
4291        // 1) ["null","int"]: Int32 (nullable)
4292        expected_cols.push(Arc::new(Int32Array::from(vec![
4293            None,
4294            Some(42),
4295            None,
4296            Some(0),
4297        ])));
4298        // 2) ["string","null"]: Utf8 (nullable)
4299        expected_cols.push(Arc::new(StringArray::from(vec![
4300            Some("s1"),
4301            None,
4302            Some("s3"),
4303            Some(""),
4304        ])));
4305        // 3) union_prim: ["boolean","int","long","float","double","bytes","string"]
4306        {
4307            let (uf, mode) = get_union("union_prim");
4308            assert!(matches!(mode, UnionMode::Dense));
4309            let generated_names: Vec<&str> = uf.iter().map(|(_, f)| f.name().as_str()).collect();
4310            let expected_names = vec![
4311                "boolean", "int", "long", "float", "double", "bytes", "string",
4312            ];
4313            assert_eq!(
4314                generated_names, expected_names,
4315                "Field names for union_prim are incorrect"
4316            );
4317            let tids = vec![
4318                tid_by_name(&uf, "long"),
4319                tid_by_name(&uf, "int"),
4320                tid_by_name(&uf, "float"),
4321                tid_by_name(&uf, "double"),
4322            ];
4323            let offs = vec![0, 0, 0, 0];
4324            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4325                "int" => Some(Arc::new(Int32Array::from(vec![-1])) as ArrayRef),
4326                "long" => Some(Arc::new(Int64Array::from(vec![1_234_567_890_123i64])) as ArrayRef),
4327                "float" => {
4328                    Some(Arc::new(arrow_array::Float32Array::from(vec![1.25f32])) as ArrayRef)
4329                }
4330                "double" => {
4331                    Some(Arc::new(arrow_array::Float64Array::from(vec![-2.5f64])) as ArrayRef)
4332                }
4333                _ => None,
4334            });
4335            expected_cols.push(arr);
4336        }
4337        // 4) union_bytes_vs_string: ["bytes","string"]
4338        {
4339            let (uf, _) = get_union("union_bytes_vs_string");
4340            let tids = vec![
4341                tid_by_name(&uf, "bytes"),
4342                tid_by_name(&uf, "string"),
4343                tid_by_name(&uf, "string"),
4344                tid_by_name(&uf, "bytes"),
4345            ];
4346            let offs = vec![0, 0, 1, 1];
4347            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4348                "bytes" => Some(
4349                    Arc::new(BinaryArray::from(vec![&[0x00, 0xFF, 0x7F][..], &[][..]])) as ArrayRef,
4350                ),
4351                "string" => Some(Arc::new(StringArray::from(vec!["hello", "world"])) as ArrayRef),
4352                _ => None,
4353            });
4354            expected_cols.push(arr);
4355        }
4356        // 5) union_fixed_dur_decfix: [Fx8, Dur12, DecFix16(decimal(10,2))]
4357        {
4358            let (uf, _) = get_union("union_fixed_dur_decfix");
4359            let tid_fx8 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(8)));
4360            let tid_dur = tid_by_dt(&uf, |dt| {
4361                matches!(
4362                    dt,
4363                    DataType::Interval(arrow_schema::IntervalUnit::MonthDayNano)
4364                )
4365            });
4366            let tid_dec = tid_by_dt(&uf, |dt| match dt {
4367                #[cfg(feature = "small_decimals")]
4368                DataType::Decimal64(10, 2) => true,
4369                DataType::Decimal128(10, 2) | DataType::Decimal256(10, 2) => true,
4370                _ => false,
4371            });
4372            let tids = vec![tid_fx8, tid_dur, tid_dec, tid_dur];
4373            let offs = vec![0, 0, 0, 1];
4374            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4375                DataType::FixedSizeBinary(8) => {
4376                    let it = [Some(fx8_a)].into_iter();
4377                    Some(Arc::new(
4378                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 8).unwrap(),
4379                    ) as ArrayRef)
4380                }
4381                DataType::Interval(IntervalUnit::MonthDayNano) => {
4382                    Some(Arc::new(arrow_array::IntervalMonthDayNanoArray::from(vec![
4383                        dur_a, dur_b,
4384                    ])) as ArrayRef)
4385                }
4386                #[cfg(feature = "small_decimals")]
4387                DataType::Decimal64(10, 2) => {
4388                    let a = arrow_array::Decimal64Array::from_iter_values([dec_fix16_neg as i64]);
4389                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4390                }
4391                DataType::Decimal128(10, 2) => {
4392                    let a = arrow_array::Decimal128Array::from_iter_values([dec_fix16_neg]);
4393                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4394                }
4395                DataType::Decimal256(10, 2) => {
4396                    let a = arrow_array::Decimal256Array::from_iter_values([i256::from_i128(
4397                        dec_fix16_neg,
4398                    )]);
4399                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4400                }
4401                _ => None,
4402            });
4403            let generated_names: Vec<&str> = uf.iter().map(|(_, f)| f.name().as_str()).collect();
4404            let expected_names = vec!["Fx8", "Dur12", "DecFix16"];
4405            assert_eq!(
4406                generated_names, expected_names,
4407                "Data type names were not generated correctly for union_fixed_dur_decfix"
4408            );
4409            expected_cols.push(arr);
4410        }
4411        // 6) union_enum_records_array_map: [enum ColorU, record RecA, record RecB, array<long>, map<string>]
4412        {
4413            let (uf, _) = get_union("union_enum_records_array_map");
4414            let tid_enum = tid_by_dt(&uf, |dt| matches!(dt, DataType::Dictionary(_, _)));
4415            let tid_reca = tid_by_dt(&uf, |dt| {
4416                if let DataType::Struct(fs) = dt {
4417                    fs.len() == 2 && fs[0].name() == "a" && fs[1].name() == "b"
4418                } else {
4419                    false
4420                }
4421            });
4422            let tid_recb = tid_by_dt(&uf, |dt| {
4423                if let DataType::Struct(fs) = dt {
4424                    fs.len() == 2 && fs[0].name() == "x" && fs[1].name() == "y"
4425                } else {
4426                    false
4427                }
4428            });
4429            let tid_arr = tid_by_dt(&uf, |dt| matches!(dt, DataType::List(_)));
4430            let tids = vec![tid_enum, tid_reca, tid_recb, tid_arr];
4431            let offs = vec![0, 0, 0, 0];
4432            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4433                DataType::Dictionary(_, _) => {
4434                    let keys = Int32Array::from(vec![0i32]); // "RED"
4435                    let values =
4436                        Arc::new(StringArray::from(vec!["RED", "GREEN", "BLUE"])) as ArrayRef;
4437                    Some(
4438                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
4439                            as ArrayRef,
4440                    )
4441                }
4442                DataType::Struct(fs)
4443                    if fs.len() == 2 && fs[0].name() == "a" && fs[1].name() == "b" =>
4444                {
4445                    let a = Int32Array::from(vec![7]);
4446                    let b = StringArray::from(vec!["x"]);
4447                    Some(Arc::new(StructArray::new(
4448                        fs.clone(),
4449                        vec![Arc::new(a), Arc::new(b)],
4450                        None,
4451                    )) as ArrayRef)
4452                }
4453                DataType::Struct(fs)
4454                    if fs.len() == 2 && fs[0].name() == "x" && fs[1].name() == "y" =>
4455                {
4456                    let x = Int64Array::from(vec![123_456_789i64]);
4457                    let y = BinaryArray::from(vec![&[0xFF, 0x00][..]]);
4458                    Some(Arc::new(StructArray::new(
4459                        fs.clone(),
4460                        vec![Arc::new(x), Arc::new(y)],
4461                        None,
4462                    )) as ArrayRef)
4463                }
4464                DataType::List(field) => {
4465                    let values = Int64Array::from(vec![1i64, 2, 3]);
4466                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3]));
4467                    Some(Arc::new(
4468                        ListArray::try_new(field.clone(), offsets, Arc::new(values), None).unwrap(),
4469                    ) as ArrayRef)
4470                }
4471                DataType::Map(_, _) => None,
4472                other => panic!("unexpected child {other:?}"),
4473            });
4474            expected_cols.push(arr);
4475        }
4476        // 7) union_date_or_fixed4: [date32, fixed(4)]
4477        {
4478            let (uf, _) = get_union("union_date_or_fixed4");
4479            let tid_date = tid_by_dt(&uf, |dt| matches!(dt, DataType::Date32));
4480            let tid_fx4 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(4)));
4481            let tids = vec![tid_date, tid_fx4, tid_date, tid_fx4];
4482            let offs = vec![0, 0, 1, 1];
4483            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4484                DataType::Date32 => {
4485                    Some(Arc::new(arrow_array::Date32Array::from(vec![date_a, 0])) as ArrayRef)
4486                }
4487                DataType::FixedSizeBinary(4) => {
4488                    let it = [Some(fx4_abcd), Some(fx4_misc)].into_iter();
4489                    Some(Arc::new(
4490                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 4).unwrap(),
4491                    ) as ArrayRef)
4492                }
4493                _ => None,
4494            });
4495            expected_cols.push(arr);
4496        }
4497        // 8) union_time_millis_or_enum: [time-millis, enum OnOff]
4498        {
4499            let (uf, _) = get_union("union_time_millis_or_enum");
4500            let tid_ms = tid_by_dt(&uf, |dt| {
4501                matches!(dt, DataType::Time32(arrow_schema::TimeUnit::Millisecond))
4502            });
4503            let tid_en = tid_by_dt(&uf, |dt| matches!(dt, DataType::Dictionary(_, _)));
4504            let tids = vec![tid_ms, tid_en, tid_en, tid_ms];
4505            let offs = vec![0, 0, 1, 1];
4506            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4507                DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
4508                    Some(Arc::new(Time32MillisecondArray::from(vec![time_ms_a, 0])) as ArrayRef)
4509                }
4510                DataType::Dictionary(_, _) => {
4511                    let keys = Int32Array::from(vec![0i32, 1]); // "ON", "OFF"
4512                    let values = Arc::new(StringArray::from(vec!["ON", "OFF"])) as ArrayRef;
4513                    Some(
4514                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
4515                            as ArrayRef,
4516                    )
4517                }
4518                _ => None,
4519            });
4520            expected_cols.push(arr);
4521        }
4522        // 9) union_time_micros_or_string: [time-micros, string]
4523        {
4524            let (uf, _) = get_union("union_time_micros_or_string");
4525            let tid_us = tid_by_dt(&uf, |dt| {
4526                matches!(dt, DataType::Time64(arrow_schema::TimeUnit::Microsecond))
4527            });
4528            let tid_s = tid_by_name(&uf, "string");
4529            let tids = vec![tid_s, tid_us, tid_s, tid_s];
4530            let offs = vec![0, 0, 1, 2];
4531            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4532                DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
4533                    Some(Arc::new(Time64MicrosecondArray::from(vec![time_us_b])) as ArrayRef)
4534                }
4535                DataType::Utf8 => {
4536                    Some(Arc::new(StringArray::from(vec!["evening", "night", ""])) as ArrayRef)
4537                }
4538                _ => None,
4539            });
4540            expected_cols.push(arr);
4541        }
4542        // 10) union_ts_millis_utc_or_array: [timestamp-millis(TZ), array<int>]
4543        {
4544            let (uf, _) = get_union("union_ts_millis_utc_or_array");
4545            let tid_ts = tid_by_dt(&uf, |dt| {
4546                matches!(
4547                    dt,
4548                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, _)
4549                )
4550            });
4551            let tid_arr = tid_by_dt(&uf, |dt| matches!(dt, DataType::List(_)));
4552            let tids = vec![tid_ts, tid_arr, tid_arr, tid_ts];
4553            let offs = vec![0, 0, 1, 1];
4554            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4555                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
4556                    let a = TimestampMillisecondArray::from(vec![
4557                        ts_ms_2024_01_01,
4558                        ts_ms_2024_01_01 + 86_400_000,
4559                    ]);
4560                    Some(Arc::new(if let Some(tz) = tz {
4561                        a.with_timezone(tz.clone())
4562                    } else {
4563                        a
4564                    }) as ArrayRef)
4565                }
4566                DataType::List(field) => {
4567                    let values = Int32Array::from(vec![0, 1, 2, -1, 0, 1]);
4568                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 6]));
4569                    Some(Arc::new(
4570                        ListArray::try_new(field.clone(), offsets, Arc::new(values), None).unwrap(),
4571                    ) as ArrayRef)
4572                }
4573                _ => None,
4574            });
4575            expected_cols.push(arr);
4576        }
4577        // 11) union_ts_micros_local_or_bytes: [local-timestamp-micros, bytes]
4578        {
4579            let (uf, _) = get_union("union_ts_micros_local_or_bytes");
4580            let tid_lts = tid_by_dt(&uf, |dt| {
4581                matches!(
4582                    dt,
4583                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None)
4584                )
4585            });
4586            let tid_b = tid_by_name(&uf, "bytes");
4587            let tids = vec![tid_b, tid_lts, tid_b, tid_b];
4588            let offs = vec![0, 0, 1, 2];
4589            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4590                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None) => Some(Arc::new(
4591                    TimestampMicrosecondArray::from(vec![ts_us_2024_01_01]),
4592                )
4593                    as ArrayRef),
4594                DataType::Binary => Some(Arc::new(BinaryArray::from(vec![
4595                    &b"\x11\x22\x33"[..],
4596                    &b"\x00"[..],
4597                    &b"\x10\x20\x30\x40"[..],
4598                ])) as ArrayRef),
4599                _ => None,
4600            });
4601            expected_cols.push(arr);
4602        }
4603        // 12) union_uuid_or_fixed10: [uuid(string)->fixed(16), fixed(10)]
4604        {
4605            let (uf, _) = get_union("union_uuid_or_fixed10");
4606            let tid_fx16 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(16)));
4607            let tid_fx10 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(10)));
4608            let tids = vec![tid_fx16, tid_fx10, tid_fx16, tid_fx10];
4609            let offs = vec![0, 0, 1, 1];
4610            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4611                DataType::FixedSizeBinary(16) => {
4612                    let it = [Some(uuid1), Some(uuid2)].into_iter();
4613                    Some(Arc::new(
4614                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
4615                    ) as ArrayRef)
4616                }
4617                DataType::FixedSizeBinary(10) => {
4618                    let it = [Some(fx10_ascii), Some(fx10_aa)].into_iter();
4619                    Some(Arc::new(
4620                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 10).unwrap(),
4621                    ) as ArrayRef)
4622                }
4623                _ => None,
4624            });
4625            expected_cols.push(arr);
4626        }
4627        // 13) union_dec_bytes_or_dec_fixed: [bytes dec(10,2), fixed(20) dec(20,4)]
4628        {
4629            let (uf, _) = get_union("union_dec_bytes_or_dec_fixed");
4630            let tid_b10s2 = tid_by_dt(&uf, |dt| match dt {
4631                #[cfg(feature = "small_decimals")]
4632                DataType::Decimal64(10, 2) => true,
4633                DataType::Decimal128(10, 2) | DataType::Decimal256(10, 2) => true,
4634                _ => false,
4635            });
4636            let tid_f20s4 = tid_by_dt(&uf, |dt| {
4637                matches!(
4638                    dt,
4639                    DataType::Decimal128(20, 4) | DataType::Decimal256(20, 4)
4640                )
4641            });
4642            let tids = vec![tid_b10s2, tid_f20s4, tid_b10s2, tid_f20s4];
4643            let offs = vec![0, 0, 1, 1];
4644            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4645                #[cfg(feature = "small_decimals")]
4646                DataType::Decimal64(10, 2) => {
4647                    let a = Decimal64Array::from_iter_values([dec_b_scale2_pos as i64, 0i64]);
4648                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4649                }
4650                DataType::Decimal128(10, 2) => {
4651                    let a = Decimal128Array::from_iter_values([dec_b_scale2_pos, 0]);
4652                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4653                }
4654                DataType::Decimal256(10, 2) => {
4655                    let a = Decimal256Array::from_iter_values([
4656                        i256::from_i128(dec_b_scale2_pos),
4657                        i256::from(0),
4658                    ]);
4659                    Some(Arc::new(a.with_precision_and_scale(10, 2).unwrap()) as ArrayRef)
4660                }
4661                DataType::Decimal128(20, 4) => {
4662                    let a = Decimal128Array::from_iter_values([dec_fix20_s4_neg, dec_fix20_s4]);
4663                    Some(Arc::new(a.with_precision_and_scale(20, 4).unwrap()) as ArrayRef)
4664                }
4665                DataType::Decimal256(20, 4) => {
4666                    let a = Decimal256Array::from_iter_values([
4667                        i256::from_i128(dec_fix20_s4_neg),
4668                        i256::from_i128(dec_fix20_s4),
4669                    ]);
4670                    Some(Arc::new(a.with_precision_and_scale(20, 4).unwrap()) as ArrayRef)
4671                }
4672                _ => None,
4673            });
4674            expected_cols.push(arr);
4675        }
4676        // 14) union_null_bytes_string: ["null","bytes","string"]
4677        {
4678            let (uf, _) = get_union("union_null_bytes_string");
4679            let tid_n = tid_by_name(&uf, "null");
4680            let tid_b = tid_by_name(&uf, "bytes");
4681            let tid_s = tid_by_name(&uf, "string");
4682            let tids = vec![tid_n, tid_b, tid_s, tid_s];
4683            let offs = vec![0, 0, 0, 1];
4684            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4685                "null" => Some(Arc::new(arrow_array::NullArray::new(1)) as ArrayRef),
4686                "bytes" => Some(Arc::new(BinaryArray::from(vec![&b"\x01\x02"[..]])) as ArrayRef),
4687                "string" => Some(Arc::new(StringArray::from(vec!["text", "u"])) as ArrayRef),
4688                _ => None,
4689            });
4690            expected_cols.push(arr);
4691        }
4692        // 15) array_of_union: array<[long,string]>
4693        {
4694            let idx = schema.index_of("array_of_union").unwrap();
4695            let dt = schema.field(idx).data_type().clone();
4696            let (item_field, _) = match &dt {
4697                DataType::List(f) => (f.clone(), ()),
4698                other => panic!("array_of_union must be List, got {other:?}"),
4699            };
4700            let (uf, _) = match item_field.data_type() {
4701                DataType::Union(f, m) => (f.clone(), m),
4702                other => panic!("array_of_union items must be Union, got {other:?}"),
4703            };
4704            let tid_l = tid_by_name(&uf, "long");
4705            let tid_s = tid_by_name(&uf, "string");
4706            let type_ids = vec![tid_l, tid_s, tid_l, tid_s, tid_l, tid_l, tid_s, tid_l];
4707            let offsets = vec![0, 0, 1, 1, 2, 3, 2, 4];
4708            let values_union =
4709                mk_dense_union(&uf, type_ids, offsets, |f| match f.name().as_str() {
4710                    "long" => {
4711                        Some(Arc::new(Int64Array::from(vec![1i64, -5, 42, -1, 0])) as ArrayRef)
4712                    }
4713                    "string" => Some(Arc::new(StringArray::from(vec!["a", "", "z"])) as ArrayRef),
4714                    _ => None,
4715                });
4716            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 5, 6, 8]));
4717            expected_cols.push(Arc::new(
4718                ListArray::try_new(item_field.clone(), list_offsets, values_union, None).unwrap(),
4719            ));
4720        }
4721        // 16) map_of_union: map<[null,double]>
4722        {
4723            let idx = schema.index_of("map_of_union").unwrap();
4724            let dt = schema.field(idx).data_type().clone();
4725            let (entry_field, ordered) = match &dt {
4726                DataType::Map(f, ordered) => (f.clone(), *ordered),
4727                other => panic!("map_of_union must be Map, got {other:?}"),
4728            };
4729            let DataType::Struct(entry_fields) = entry_field.data_type() else {
4730                panic!("map entries must be struct")
4731            };
4732            let key_field = entry_fields[0].clone();
4733            let val_field = entry_fields[1].clone();
4734            let keys = StringArray::from(vec!["a", "b", "x", "pi"]);
4735            let rounded_pi = (std::f64::consts::PI * 100_000.0).round() / 100_000.0;
4736            let values: ArrayRef = match val_field.data_type() {
4737                DataType::Union(uf, _) => {
4738                    let tid_n = tid_by_name(uf, "null");
4739                    let tid_d = tid_by_name(uf, "double");
4740                    let tids = vec![tid_n, tid_d, tid_d, tid_d];
4741                    let offs = vec![0, 0, 1, 2];
4742                    mk_dense_union(uf, tids, offs, |f| match f.name().as_str() {
4743                        "null" => Some(Arc::new(NullArray::new(1)) as ArrayRef),
4744                        "double" => Some(Arc::new(arrow_array::Float64Array::from(vec![
4745                            2.5f64, -0.5f64, rounded_pi,
4746                        ])) as ArrayRef),
4747                        _ => None,
4748                    })
4749                }
4750                DataType::Float64 => Arc::new(arrow_array::Float64Array::from(vec![
4751                    None,
4752                    Some(2.5),
4753                    Some(-0.5),
4754                    Some(rounded_pi),
4755                ])),
4756                other => panic!("unexpected map value type {other:?}"),
4757            };
4758            let entries = StructArray::new(
4759                Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
4760                vec![Arc::new(keys) as ArrayRef, values],
4761                None,
4762            );
4763            let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3, 3, 4]));
4764            expected_cols.push(Arc::new(MapArray::new(
4765                entry_field,
4766                offsets,
4767                entries,
4768                None,
4769                ordered,
4770            )));
4771        }
4772        // 17) record_with_union_field: struct { id:int, u:[int,string] }
4773        {
4774            let idx = schema.index_of("record_with_union_field").unwrap();
4775            let DataType::Struct(rec_fields) = schema.field(idx).data_type() else {
4776                panic!("record_with_union_field should be Struct")
4777            };
4778            let id = Int32Array::from(vec![1, 2, 3, 4]);
4779            let u_field = rec_fields.iter().find(|f| f.name() == "u").unwrap();
4780            let DataType::Union(uf, _) = u_field.data_type() else {
4781                panic!("u must be Union")
4782            };
4783            let tid_i = tid_by_name(uf, "int");
4784            let tid_s = tid_by_name(uf, "string");
4785            let tids = vec![tid_s, tid_i, tid_i, tid_s];
4786            let offs = vec![0, 0, 1, 1];
4787            let u = mk_dense_union(uf, tids, offs, |f| match f.name().as_str() {
4788                "int" => Some(Arc::new(Int32Array::from(vec![99, 0])) as ArrayRef),
4789                "string" => Some(Arc::new(StringArray::from(vec!["one", "four"])) as ArrayRef),
4790                _ => None,
4791            });
4792            let rec = StructArray::new(rec_fields.clone(), vec![Arc::new(id) as ArrayRef, u], None);
4793            expected_cols.push(Arc::new(rec));
4794        }
4795        // 18) union_ts_micros_utc_or_map: [timestamp-micros(TZ), map<long>]
4796        {
4797            let (uf, _) = get_union("union_ts_micros_utc_or_map");
4798            let tid_ts = tid_by_dt(&uf, |dt| {
4799                matches!(
4800                    dt,
4801                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, Some(_))
4802                )
4803            });
4804            let tid_map = tid_by_dt(&uf, |dt| matches!(dt, DataType::Map(_, _)));
4805            let tids = vec![tid_ts, tid_map, tid_ts, tid_map];
4806            let offs = vec![0, 0, 1, 1];
4807            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4808                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
4809                    let a = TimestampMicrosecondArray::from(vec![ts_us_2024_01_01, 0i64]);
4810                    Some(Arc::new(if let Some(tz) = tz {
4811                        a.with_timezone(tz.clone())
4812                    } else {
4813                        a
4814                    }) as ArrayRef)
4815                }
4816                DataType::Map(entry_field, ordered) => {
4817                    let DataType::Struct(fs) = entry_field.data_type() else {
4818                        panic!("map entries must be struct")
4819                    };
4820                    let key_field = fs[0].clone();
4821                    let val_field = fs[1].clone();
4822                    assert_eq!(key_field.data_type(), &DataType::Utf8);
4823                    assert_eq!(val_field.data_type(), &DataType::Int64);
4824                    let keys = StringArray::from(vec!["k1", "k2", "n"]);
4825                    let vals = Int64Array::from(vec![1i64, 2, 0]);
4826                    let entries = StructArray::new(
4827                        Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
4828                        vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
4829                        None,
4830                    );
4831                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3]));
4832                    Some(Arc::new(MapArray::new(
4833                        entry_field.clone(),
4834                        offsets,
4835                        entries,
4836                        None,
4837                        *ordered,
4838                    )) as ArrayRef)
4839                }
4840                _ => None,
4841            });
4842            expected_cols.push(arr);
4843        }
4844        // 19) union_ts_millis_local_or_string: [local-timestamp-millis, string]
4845        {
4846            let (uf, _) = get_union("union_ts_millis_local_or_string");
4847            let tid_ts = tid_by_dt(&uf, |dt| {
4848                matches!(
4849                    dt,
4850                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None)
4851                )
4852            });
4853            let tid_s = tid_by_name(&uf, "string");
4854            let tids = vec![tid_s, tid_ts, tid_s, tid_s];
4855            let offs = vec![0, 0, 1, 2];
4856            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
4857                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None) => Some(Arc::new(
4858                    TimestampMillisecondArray::from(vec![ts_ms_2024_01_01]),
4859                )
4860                    as ArrayRef),
4861                DataType::Utf8 => {
4862                    Some(
4863                        Arc::new(StringArray::from(vec!["local midnight", "done", ""])) as ArrayRef,
4864                    )
4865                }
4866                _ => None,
4867            });
4868            expected_cols.push(arr);
4869        }
4870        // 20) union_bool_or_string: ["boolean","string"]
4871        {
4872            let (uf, _) = get_union("union_bool_or_string");
4873            let tid_b = tid_by_name(&uf, "boolean");
4874            let tid_s = tid_by_name(&uf, "string");
4875            let tids = vec![tid_b, tid_s, tid_b, tid_s];
4876            let offs = vec![0, 0, 1, 1];
4877            let arr = mk_dense_union(&uf, tids, offs, |f| match f.name().as_str() {
4878                "boolean" => Some(Arc::new(BooleanArray::from(vec![true, false])) as ArrayRef),
4879                "string" => Some(Arc::new(StringArray::from(vec!["no", "yes"])) as ArrayRef),
4880                _ => None,
4881            });
4882            expected_cols.push(arr);
4883        }
4884        let expected = RecordBatch::try_new(schema.clone(), expected_cols).unwrap();
4885        assert_eq!(
4886            actual, expected,
4887            "full end-to-end equality for union_fields.avro"
4888        );
4889    }
4890
4891    #[test]
4892    fn test_read_zero_byte_avro_file() {
4893        let batch = read_file("test/data/zero_byte.avro", 3, false);
4894        let schema = batch.schema();
4895        assert_eq!(schema.fields().len(), 1);
4896        let field = schema.field(0);
4897        assert_eq!(field.name(), "data");
4898        assert_eq!(field.data_type(), &DataType::Binary);
4899        assert!(field.is_nullable());
4900        assert_eq!(batch.num_rows(), 3);
4901        assert_eq!(batch.num_columns(), 1);
4902        let binary_array = batch
4903            .column(0)
4904            .as_any()
4905            .downcast_ref::<BinaryArray>()
4906            .unwrap();
4907        assert!(binary_array.is_null(0));
4908        assert!(binary_array.is_valid(1));
4909        assert_eq!(binary_array.value(1), b"");
4910        assert!(binary_array.is_valid(2));
4911        assert_eq!(binary_array.value(2), b"some bytes");
4912    }
4913
4914    #[test]
4915    fn test_alltypes() {
4916        let expected = RecordBatch::try_from_iter_with_nullable([
4917            (
4918                "id",
4919                Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _,
4920                true,
4921            ),
4922            (
4923                "bool_col",
4924                Arc::new(BooleanArray::from_iter((0..8).map(|x| Some(x % 2 == 0)))) as _,
4925                true,
4926            ),
4927            (
4928                "tinyint_col",
4929                Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
4930                true,
4931            ),
4932            (
4933                "smallint_col",
4934                Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
4935                true,
4936            ),
4937            (
4938                "int_col",
4939                Arc::new(Int32Array::from_iter_values((0..8).map(|x| x % 2))) as _,
4940                true,
4941            ),
4942            (
4943                "bigint_col",
4944                Arc::new(Int64Array::from_iter_values((0..8).map(|x| (x % 2) * 10))) as _,
4945                true,
4946            ),
4947            (
4948                "float_col",
4949                Arc::new(Float32Array::from_iter_values(
4950                    (0..8).map(|x| (x % 2) as f32 * 1.1),
4951                )) as _,
4952                true,
4953            ),
4954            (
4955                "double_col",
4956                Arc::new(Float64Array::from_iter_values(
4957                    (0..8).map(|x| (x % 2) as f64 * 10.1),
4958                )) as _,
4959                true,
4960            ),
4961            (
4962                "date_string_col",
4963                Arc::new(BinaryArray::from_iter_values([
4964                    [48, 51, 47, 48, 49, 47, 48, 57],
4965                    [48, 51, 47, 48, 49, 47, 48, 57],
4966                    [48, 52, 47, 48, 49, 47, 48, 57],
4967                    [48, 52, 47, 48, 49, 47, 48, 57],
4968                    [48, 50, 47, 48, 49, 47, 48, 57],
4969                    [48, 50, 47, 48, 49, 47, 48, 57],
4970                    [48, 49, 47, 48, 49, 47, 48, 57],
4971                    [48, 49, 47, 48, 49, 47, 48, 57],
4972                ])) as _,
4973                true,
4974            ),
4975            (
4976                "string_col",
4977                Arc::new(BinaryArray::from_iter_values((0..8).map(|x| [48 + x % 2]))) as _,
4978                true,
4979            ),
4980            (
4981                "timestamp_col",
4982                Arc::new(
4983                    TimestampMicrosecondArray::from_iter_values([
4984                        1235865600000000, // 2009-03-01T00:00:00.000
4985                        1235865660000000, // 2009-03-01T00:01:00.000
4986                        1238544000000000, // 2009-04-01T00:00:00.000
4987                        1238544060000000, // 2009-04-01T00:01:00.000
4988                        1233446400000000, // 2009-02-01T00:00:00.000
4989                        1233446460000000, // 2009-02-01T00:01:00.000
4990                        1230768000000000, // 2009-01-01T00:00:00.000
4991                        1230768060000000, // 2009-01-01T00:01:00.000
4992                    ])
4993                    .with_timezone("+00:00"),
4994                ) as _,
4995                true,
4996            ),
4997        ])
4998        .unwrap();
4999
5000        for file in files() {
5001            let file = arrow_test_data(file);
5002
5003            assert_eq!(read_file(&file, 8, false), expected);
5004            assert_eq!(read_file(&file, 3, false), expected);
5005        }
5006    }
5007
5008    #[test]
5009    // TODO: avoid requiring snappy for this file
5010    #[cfg(feature = "snappy")]
5011    fn test_alltypes_dictionary() {
5012        let file = "avro/alltypes_dictionary.avro";
5013        let expected = RecordBatch::try_from_iter_with_nullable([
5014            ("id", Arc::new(Int32Array::from(vec![0, 1])) as _, true),
5015            (
5016                "bool_col",
5017                Arc::new(BooleanArray::from(vec![Some(true), Some(false)])) as _,
5018                true,
5019            ),
5020            (
5021                "tinyint_col",
5022                Arc::new(Int32Array::from(vec![0, 1])) as _,
5023                true,
5024            ),
5025            (
5026                "smallint_col",
5027                Arc::new(Int32Array::from(vec![0, 1])) as _,
5028                true,
5029            ),
5030            ("int_col", Arc::new(Int32Array::from(vec![0, 1])) as _, true),
5031            (
5032                "bigint_col",
5033                Arc::new(Int64Array::from(vec![0, 10])) as _,
5034                true,
5035            ),
5036            (
5037                "float_col",
5038                Arc::new(Float32Array::from(vec![0.0, 1.1])) as _,
5039                true,
5040            ),
5041            (
5042                "double_col",
5043                Arc::new(Float64Array::from(vec![0.0, 10.1])) as _,
5044                true,
5045            ),
5046            (
5047                "date_string_col",
5048                Arc::new(BinaryArray::from_iter_values([b"01/01/09", b"01/01/09"])) as _,
5049                true,
5050            ),
5051            (
5052                "string_col",
5053                Arc::new(BinaryArray::from_iter_values([b"0", b"1"])) as _,
5054                true,
5055            ),
5056            (
5057                "timestamp_col",
5058                Arc::new(
5059                    TimestampMicrosecondArray::from_iter_values([
5060                        1230768000000000, // 2009-01-01T00:00:00.000
5061                        1230768060000000, // 2009-01-01T00:01:00.000
5062                    ])
5063                    .with_timezone("+00:00"),
5064                ) as _,
5065                true,
5066            ),
5067        ])
5068        .unwrap();
5069        let file_path = arrow_test_data(file);
5070        let batch_large = read_file(&file_path, 8, false);
5071        assert_eq!(
5072            batch_large, expected,
5073            "Decoded RecordBatch does not match for file {file}"
5074        );
5075        let batch_small = read_file(&file_path, 3, false);
5076        assert_eq!(
5077            batch_small, expected,
5078            "Decoded RecordBatch (batch size 3) does not match for file {file}"
5079        );
5080    }
5081
5082    #[test]
5083    fn test_alltypes_nulls_plain() {
5084        let file = "avro/alltypes_nulls_plain.avro";
5085        let expected = RecordBatch::try_from_iter_with_nullable([
5086            (
5087                "string_col",
5088                Arc::new(StringArray::from(vec![None::<&str>])) as _,
5089                true,
5090            ),
5091            ("int_col", Arc::new(Int32Array::from(vec![None])) as _, true),
5092            (
5093                "bool_col",
5094                Arc::new(BooleanArray::from(vec![None])) as _,
5095                true,
5096            ),
5097            (
5098                "bigint_col",
5099                Arc::new(Int64Array::from(vec![None])) as _,
5100                true,
5101            ),
5102            (
5103                "float_col",
5104                Arc::new(Float32Array::from(vec![None])) as _,
5105                true,
5106            ),
5107            (
5108                "double_col",
5109                Arc::new(Float64Array::from(vec![None])) as _,
5110                true,
5111            ),
5112            (
5113                "bytes_col",
5114                Arc::new(BinaryArray::from(vec![None::<&[u8]>])) as _,
5115                true,
5116            ),
5117        ])
5118        .unwrap();
5119        let file_path = arrow_test_data(file);
5120        let batch_large = read_file(&file_path, 8, false);
5121        assert_eq!(
5122            batch_large, expected,
5123            "Decoded RecordBatch does not match for file {file}"
5124        );
5125        let batch_small = read_file(&file_path, 3, false);
5126        assert_eq!(
5127            batch_small, expected,
5128            "Decoded RecordBatch (batch size 3) does not match for file {file}"
5129        );
5130    }
5131
5132    #[test]
5133    // TODO: avoid requiring snappy for this file
5134    #[cfg(feature = "snappy")]
5135    fn test_binary() {
5136        let file = arrow_test_data("avro/binary.avro");
5137        let batch = read_file(&file, 8, false);
5138        let expected = RecordBatch::try_from_iter_with_nullable([(
5139            "foo",
5140            Arc::new(BinaryArray::from_iter_values(vec![
5141                b"\x00" as &[u8],
5142                b"\x01" as &[u8],
5143                b"\x02" as &[u8],
5144                b"\x03" as &[u8],
5145                b"\x04" as &[u8],
5146                b"\x05" as &[u8],
5147                b"\x06" as &[u8],
5148                b"\x07" as &[u8],
5149                b"\x08" as &[u8],
5150                b"\t" as &[u8],
5151                b"\n" as &[u8],
5152                b"\x0b" as &[u8],
5153            ])) as Arc<dyn Array>,
5154            true,
5155        )])
5156        .unwrap();
5157        assert_eq!(batch, expected);
5158    }
5159
5160    #[test]
5161    // TODO: avoid requiring snappy for these files
5162    #[cfg(feature = "snappy")]
5163    fn test_decimal() {
5164        // Choose expected Arrow types depending on the `small_decimals` feature flag.
5165        // With `small_decimals` enabled, Decimal32/Decimal64 are used where their
5166        // precision allows; otherwise, those cases resolve to Decimal128.
5167        #[cfg(feature = "small_decimals")]
5168        let files: [(&str, DataType, HashMap<String, String>); 8] = [
5169            (
5170                "avro/fixed_length_decimal.avro",
5171                DataType::Decimal128(25, 2),
5172                HashMap::from([
5173                    (
5174                        "avro.namespace".to_string(),
5175                        "topLevelRecord.value".to_string(),
5176                    ),
5177                    ("avro.name".to_string(), "fixed".to_string()),
5178                ]),
5179            ),
5180            (
5181                "avro/fixed_length_decimal_legacy.avro",
5182                DataType::Decimal64(13, 2),
5183                HashMap::from([
5184                    (
5185                        "avro.namespace".to_string(),
5186                        "topLevelRecord.value".to_string(),
5187                    ),
5188                    ("avro.name".to_string(), "fixed".to_string()),
5189                ]),
5190            ),
5191            (
5192                "avro/int32_decimal.avro",
5193                DataType::Decimal32(4, 2),
5194                HashMap::from([
5195                    (
5196                        "avro.namespace".to_string(),
5197                        "topLevelRecord.value".to_string(),
5198                    ),
5199                    ("avro.name".to_string(), "fixed".to_string()),
5200                ]),
5201            ),
5202            (
5203                "avro/int64_decimal.avro",
5204                DataType::Decimal64(10, 2),
5205                HashMap::from([
5206                    (
5207                        "avro.namespace".to_string(),
5208                        "topLevelRecord.value".to_string(),
5209                    ),
5210                    ("avro.name".to_string(), "fixed".to_string()),
5211                ]),
5212            ),
5213            (
5214                "test/data/int256_decimal.avro",
5215                DataType::Decimal256(76, 10),
5216                HashMap::new(),
5217            ),
5218            (
5219                "test/data/fixed256_decimal.avro",
5220                DataType::Decimal256(76, 10),
5221                HashMap::from([("avro.name".to_string(), "Decimal256Fixed".to_string())]),
5222            ),
5223            (
5224                "test/data/fixed_length_decimal_legacy_32.avro",
5225                DataType::Decimal32(9, 2),
5226                HashMap::from([("avro.name".to_string(), "Decimal32FixedLegacy".to_string())]),
5227            ),
5228            (
5229                "test/data/int128_decimal.avro",
5230                DataType::Decimal128(38, 2),
5231                HashMap::new(),
5232            ),
5233        ];
5234        #[cfg(not(feature = "small_decimals"))]
5235        let files: [(&str, DataType, HashMap<String, String>); 8] = [
5236            (
5237                "avro/fixed_length_decimal.avro",
5238                DataType::Decimal128(25, 2),
5239                HashMap::from([
5240                    (
5241                        "avro.namespace".to_string(),
5242                        "topLevelRecord.value".to_string(),
5243                    ),
5244                    ("avro.name".to_string(), "fixed".to_string()),
5245                ]),
5246            ),
5247            (
5248                "avro/fixed_length_decimal_legacy.avro",
5249                DataType::Decimal128(13, 2),
5250                HashMap::from([
5251                    (
5252                        "avro.namespace".to_string(),
5253                        "topLevelRecord.value".to_string(),
5254                    ),
5255                    ("avro.name".to_string(), "fixed".to_string()),
5256                ]),
5257            ),
5258            (
5259                "avro/int32_decimal.avro",
5260                DataType::Decimal128(4, 2),
5261                HashMap::from([
5262                    (
5263                        "avro.namespace".to_string(),
5264                        "topLevelRecord.value".to_string(),
5265                    ),
5266                    ("avro.name".to_string(), "fixed".to_string()),
5267                ]),
5268            ),
5269            (
5270                "avro/int64_decimal.avro",
5271                DataType::Decimal128(10, 2),
5272                HashMap::from([
5273                    (
5274                        "avro.namespace".to_string(),
5275                        "topLevelRecord.value".to_string(),
5276                    ),
5277                    ("avro.name".to_string(), "fixed".to_string()),
5278                ]),
5279            ),
5280            (
5281                "test/data/int256_decimal.avro",
5282                DataType::Decimal256(76, 10),
5283                HashMap::new(),
5284            ),
5285            (
5286                "test/data/fixed256_decimal.avro",
5287                DataType::Decimal256(76, 10),
5288                HashMap::from([("avro.name".to_string(), "Decimal256Fixed".to_string())]),
5289            ),
5290            (
5291                "test/data/fixed_length_decimal_legacy_32.avro",
5292                DataType::Decimal128(9, 2),
5293                HashMap::from([("avro.name".to_string(), "Decimal32FixedLegacy".to_string())]),
5294            ),
5295            (
5296                "test/data/int128_decimal.avro",
5297                DataType::Decimal128(38, 2),
5298                HashMap::new(),
5299            ),
5300        ];
5301        for (file, expected_dt, mut metadata) in files {
5302            let (precision, scale) = match expected_dt {
5303                DataType::Decimal32(p, s)
5304                | DataType::Decimal64(p, s)
5305                | DataType::Decimal128(p, s)
5306                | DataType::Decimal256(p, s) => (p, s),
5307                _ => unreachable!("Unexpected decimal type in test inputs"),
5308            };
5309            assert!(scale >= 0, "test data uses non-negative scales only");
5310            let scale_u32 = scale as u32;
5311            let file_path: String = if file.starts_with("avro/") {
5312                arrow_test_data(file)
5313            } else {
5314                std::path::PathBuf::from(env!("CARGO_MANIFEST_DIR"))
5315                    .join(file)
5316                    .to_string_lossy()
5317                    .into_owned()
5318            };
5319            let pow10: i128 = 10i128.pow(scale_u32);
5320            let values_i128: Vec<i128> = (1..=24).map(|n| (n as i128) * pow10).collect();
5321            let build_expected = |dt: &DataType, values: &[i128]| -> ArrayRef {
5322                match *dt {
5323                    #[cfg(feature = "small_decimals")]
5324                    DataType::Decimal32(p, s) => {
5325                        let it = values.iter().map(|&v| v as i32);
5326                        Arc::new(
5327                            Decimal32Array::from_iter_values(it)
5328                                .with_precision_and_scale(p, s)
5329                                .unwrap(),
5330                        )
5331                    }
5332                    #[cfg(feature = "small_decimals")]
5333                    DataType::Decimal64(p, s) => {
5334                        let it = values.iter().map(|&v| v as i64);
5335                        Arc::new(
5336                            Decimal64Array::from_iter_values(it)
5337                                .with_precision_and_scale(p, s)
5338                                .unwrap(),
5339                        )
5340                    }
5341                    DataType::Decimal128(p, s) => {
5342                        let it = values.iter().copied();
5343                        Arc::new(
5344                            Decimal128Array::from_iter_values(it)
5345                                .with_precision_and_scale(p, s)
5346                                .unwrap(),
5347                        )
5348                    }
5349                    DataType::Decimal256(p, s) => {
5350                        let it = values.iter().map(|&v| i256::from_i128(v));
5351                        Arc::new(
5352                            Decimal256Array::from_iter_values(it)
5353                                .with_precision_and_scale(p, s)
5354                                .unwrap(),
5355                        )
5356                    }
5357                    _ => unreachable!("Unexpected decimal type in test"),
5358                }
5359            };
5360            let actual_batch = read_file(&file_path, 8, false);
5361            let actual_nullable = actual_batch.schema().field(0).is_nullable();
5362            let expected_array = build_expected(&expected_dt, &values_i128);
5363            metadata.insert("precision".to_string(), precision.to_string());
5364            metadata.insert("scale".to_string(), scale.to_string());
5365            let field =
5366                Field::new("value", expected_dt.clone(), actual_nullable).with_metadata(metadata);
5367            let expected_schema = Arc::new(Schema::new(vec![field]));
5368            let expected_batch =
5369                RecordBatch::try_new(expected_schema.clone(), vec![expected_array]).unwrap();
5370            assert_eq!(
5371                actual_batch, expected_batch,
5372                "Decoded RecordBatch does not match for {file}"
5373            );
5374            let actual_batch_small = read_file(&file_path, 3, false);
5375            assert_eq!(
5376                actual_batch_small, expected_batch,
5377                "Decoded RecordBatch does not match for {file} with batch size 3"
5378            );
5379        }
5380    }
5381
5382    #[test]
5383    fn test_read_duration_logical_types_feature_toggle() -> Result<(), ArrowError> {
5384        let file_path = std::path::PathBuf::from(env!("CARGO_MANIFEST_DIR"))
5385            .join("test/data/duration_logical_types.avro")
5386            .to_string_lossy()
5387            .into_owned();
5388
5389        let actual_batch = read_file(&file_path, 4, false);
5390
5391        let expected_batch = {
5392            #[cfg(feature = "avro_custom_types")]
5393            {
5394                let schema = Arc::new(Schema::new(vec![
5395                    Field::new(
5396                        "duration_time_nanos",
5397                        DataType::Duration(TimeUnit::Nanosecond),
5398                        false,
5399                    ),
5400                    Field::new(
5401                        "duration_time_micros",
5402                        DataType::Duration(TimeUnit::Microsecond),
5403                        false,
5404                    ),
5405                    Field::new(
5406                        "duration_time_millis",
5407                        DataType::Duration(TimeUnit::Millisecond),
5408                        false,
5409                    ),
5410                    Field::new(
5411                        "duration_time_seconds",
5412                        DataType::Duration(TimeUnit::Second),
5413                        false,
5414                    ),
5415                ]));
5416
5417                let nanos = Arc::new(PrimitiveArray::<DurationNanosecondType>::from(vec![
5418                    10, 20, 30, 40,
5419                ])) as ArrayRef;
5420                let micros = Arc::new(PrimitiveArray::<DurationMicrosecondType>::from(vec![
5421                    100, 200, 300, 400,
5422                ])) as ArrayRef;
5423                let millis = Arc::new(PrimitiveArray::<DurationMillisecondType>::from(vec![
5424                    1000, 2000, 3000, 4000,
5425                ])) as ArrayRef;
5426                let seconds = Arc::new(PrimitiveArray::<DurationSecondType>::from(vec![1, 2, 3, 4]))
5427                    as ArrayRef;
5428
5429                RecordBatch::try_new(schema, vec![nanos, micros, millis, seconds])?
5430            }
5431            #[cfg(not(feature = "avro_custom_types"))]
5432            {
5433                let schema = Arc::new(Schema::new(vec![
5434                    Field::new("duration_time_nanos", DataType::Int64, false).with_metadata(
5435                        [(
5436                            "logicalType".to_string(),
5437                            "arrow.duration-nanos".to_string(),
5438                        )]
5439                        .into(),
5440                    ),
5441                    Field::new("duration_time_micros", DataType::Int64, false).with_metadata(
5442                        [(
5443                            "logicalType".to_string(),
5444                            "arrow.duration-micros".to_string(),
5445                        )]
5446                        .into(),
5447                    ),
5448                    Field::new("duration_time_millis", DataType::Int64, false).with_metadata(
5449                        [(
5450                            "logicalType".to_string(),
5451                            "arrow.duration-millis".to_string(),
5452                        )]
5453                        .into(),
5454                    ),
5455                    Field::new("duration_time_seconds", DataType::Int64, false).with_metadata(
5456                        [(
5457                            "logicalType".to_string(),
5458                            "arrow.duration-seconds".to_string(),
5459                        )]
5460                        .into(),
5461                    ),
5462                ]));
5463
5464                let nanos =
5465                    Arc::new(PrimitiveArray::<Int64Type>::from(vec![10, 20, 30, 40])) as ArrayRef;
5466                let micros = Arc::new(PrimitiveArray::<Int64Type>::from(vec![100, 200, 300, 400]))
5467                    as ArrayRef;
5468                let millis = Arc::new(PrimitiveArray::<Int64Type>::from(vec![
5469                    1000, 2000, 3000, 4000,
5470                ])) as ArrayRef;
5471                let seconds =
5472                    Arc::new(PrimitiveArray::<Int64Type>::from(vec![1, 2, 3, 4])) as ArrayRef;
5473
5474                RecordBatch::try_new(schema, vec![nanos, micros, millis, seconds])?
5475            }
5476        };
5477
5478        assert_eq!(actual_batch, expected_batch);
5479
5480        Ok(())
5481    }
5482
5483    #[test]
5484    // TODO: avoid requiring snappy for this file
5485    #[cfg(feature = "snappy")]
5486    fn test_dict_pages_offset_zero() {
5487        let file = arrow_test_data("avro/dict-page-offset-zero.avro");
5488        let batch = read_file(&file, 32, false);
5489        let num_rows = batch.num_rows();
5490        let expected_field = Int32Array::from(vec![Some(1552); num_rows]);
5491        let expected = RecordBatch::try_from_iter_with_nullable([(
5492            "l_partkey",
5493            Arc::new(expected_field) as Arc<dyn Array>,
5494            true,
5495        )])
5496        .unwrap();
5497        assert_eq!(batch, expected);
5498    }
5499
5500    #[test]
5501    // TODO: avoid requiring snappy for this file
5502    #[cfg(feature = "snappy")]
5503    fn test_list_columns() {
5504        let file = arrow_test_data("avro/list_columns.avro");
5505        let mut int64_list_builder = ListBuilder::new(Int64Builder::new());
5506        {
5507            {
5508                let values = int64_list_builder.values();
5509                values.append_value(1);
5510                values.append_value(2);
5511                values.append_value(3);
5512            }
5513            int64_list_builder.append(true);
5514        }
5515        {
5516            {
5517                let values = int64_list_builder.values();
5518                values.append_null();
5519                values.append_value(1);
5520            }
5521            int64_list_builder.append(true);
5522        }
5523        {
5524            {
5525                let values = int64_list_builder.values();
5526                values.append_value(4);
5527            }
5528            int64_list_builder.append(true);
5529        }
5530        let int64_list = int64_list_builder.finish();
5531        let mut utf8_list_builder = ListBuilder::new(StringBuilder::new());
5532        {
5533            {
5534                let values = utf8_list_builder.values();
5535                values.append_value("abc");
5536                values.append_value("efg");
5537                values.append_value("hij");
5538            }
5539            utf8_list_builder.append(true);
5540        }
5541        {
5542            utf8_list_builder.append(false);
5543        }
5544        {
5545            {
5546                let values = utf8_list_builder.values();
5547                values.append_value("efg");
5548                values.append_null();
5549                values.append_value("hij");
5550                values.append_value("xyz");
5551            }
5552            utf8_list_builder.append(true);
5553        }
5554        let utf8_list = utf8_list_builder.finish();
5555        let expected = RecordBatch::try_from_iter_with_nullable([
5556            ("int64_list", Arc::new(int64_list) as Arc<dyn Array>, true),
5557            ("utf8_list", Arc::new(utf8_list) as Arc<dyn Array>, true),
5558        ])
5559        .unwrap();
5560        let batch = read_file(&file, 8, false);
5561        assert_eq!(batch, expected);
5562    }
5563
5564    #[test]
5565    #[cfg(feature = "snappy")]
5566    fn test_nested_lists() {
5567        use arrow_data::ArrayDataBuilder;
5568        let file = arrow_test_data("avro/nested_lists.snappy.avro");
5569        let inner_values = StringArray::from(vec![
5570            Some("a"),
5571            Some("b"),
5572            Some("c"),
5573            Some("d"),
5574            Some("a"),
5575            Some("b"),
5576            Some("c"),
5577            Some("d"),
5578            Some("e"),
5579            Some("a"),
5580            Some("b"),
5581            Some("c"),
5582            Some("d"),
5583            Some("e"),
5584            Some("f"),
5585        ]);
5586        let inner_offsets = Buffer::from_slice_ref([0, 2, 3, 3, 4, 6, 8, 8, 9, 11, 13, 14, 14, 15]);
5587        let inner_validity = [
5588            true, true, false, true, true, true, false, true, true, true, true, false, true,
5589        ];
5590        let inner_null_buffer = Buffer::from_iter(inner_validity.iter().copied());
5591        let inner_field = Field::new("item", DataType::Utf8, true);
5592        let inner_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(inner_field)))
5593            .len(13)
5594            .add_buffer(inner_offsets)
5595            .add_child_data(inner_values.to_data())
5596            .null_bit_buffer(Some(inner_null_buffer))
5597            .build()
5598            .unwrap();
5599        let inner_list_array = ListArray::from(inner_list_data);
5600        let middle_offsets = Buffer::from_slice_ref([0, 2, 4, 6, 8, 11, 13]);
5601        let middle_validity = [true; 6];
5602        let middle_null_buffer = Buffer::from_iter(middle_validity.iter().copied());
5603        let middle_field = Field::new("item", inner_list_array.data_type().clone(), true);
5604        let middle_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(middle_field)))
5605            .len(6)
5606            .add_buffer(middle_offsets)
5607            .add_child_data(inner_list_array.to_data())
5608            .null_bit_buffer(Some(middle_null_buffer))
5609            .build()
5610            .unwrap();
5611        let middle_list_array = ListArray::from(middle_list_data);
5612        let outer_offsets = Buffer::from_slice_ref([0, 2, 4, 6]);
5613        let outer_null_buffer = Buffer::from_slice_ref([0b111]); // all 3 rows valid
5614        let outer_field = Field::new("item", middle_list_array.data_type().clone(), true);
5615        let outer_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(outer_field)))
5616            .len(3)
5617            .add_buffer(outer_offsets)
5618            .add_child_data(middle_list_array.to_data())
5619            .null_bit_buffer(Some(outer_null_buffer))
5620            .build()
5621            .unwrap();
5622        let a_expected = ListArray::from(outer_list_data);
5623        let b_expected = Int32Array::from(vec![1, 1, 1]);
5624        let expected = RecordBatch::try_from_iter_with_nullable([
5625            ("a", Arc::new(a_expected) as Arc<dyn Array>, true),
5626            ("b", Arc::new(b_expected) as Arc<dyn Array>, true),
5627        ])
5628        .unwrap();
5629        let left = read_file(&file, 8, false);
5630        assert_eq!(left, expected, "Mismatch for batch size=8");
5631        let left_small = read_file(&file, 3, false);
5632        assert_eq!(left_small, expected, "Mismatch for batch size=3");
5633    }
5634
5635    #[test]
5636    fn test_simple() {
5637        let tests = [
5638            ("avro/simple_enum.avro", 4, build_expected_enum(), 2),
5639            ("avro/simple_fixed.avro", 2, build_expected_fixed(), 1),
5640        ];
5641
5642        fn build_expected_enum() -> RecordBatch {
5643            // Build the DictionaryArrays for f1, f2, f3
5644            let keys_f1 = Int32Array::from(vec![0, 1, 2, 3]);
5645            let vals_f1 = StringArray::from(vec!["a", "b", "c", "d"]);
5646            let f1_dict =
5647                DictionaryArray::<Int32Type>::try_new(keys_f1, Arc::new(vals_f1)).unwrap();
5648            let keys_f2 = Int32Array::from(vec![2, 3, 0, 1]);
5649            let vals_f2 = StringArray::from(vec!["e", "f", "g", "h"]);
5650            let f2_dict =
5651                DictionaryArray::<Int32Type>::try_new(keys_f2, Arc::new(vals_f2)).unwrap();
5652            let keys_f3 = Int32Array::from(vec![Some(1), Some(2), None, Some(0)]);
5653            let vals_f3 = StringArray::from(vec!["i", "j", "k"]);
5654            let f3_dict =
5655                DictionaryArray::<Int32Type>::try_new(keys_f3, Arc::new(vals_f3)).unwrap();
5656            let dict_type =
5657                DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8));
5658            let mut md_f1 = HashMap::new();
5659            md_f1.insert(
5660                AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
5661                r#"["a","b","c","d"]"#.to_string(),
5662            );
5663            md_f1.insert(AVRO_NAME_METADATA_KEY.to_string(), "enum1".to_string());
5664            md_f1.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns1".to_string());
5665            let f1_field = Field::new("f1", dict_type.clone(), false).with_metadata(md_f1);
5666            let mut md_f2 = HashMap::new();
5667            md_f2.insert(
5668                AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
5669                r#"["e","f","g","h"]"#.to_string(),
5670            );
5671            md_f2.insert(AVRO_NAME_METADATA_KEY.to_string(), "enum2".to_string());
5672            md_f2.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns2".to_string());
5673            let f2_field = Field::new("f2", dict_type.clone(), false).with_metadata(md_f2);
5674            let mut md_f3 = HashMap::new();
5675            md_f3.insert(
5676                AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
5677                r#"["i","j","k"]"#.to_string(),
5678            );
5679            md_f3.insert(AVRO_NAME_METADATA_KEY.to_string(), "enum3".to_string());
5680            md_f3.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns1".to_string());
5681            let f3_field = Field::new("f3", dict_type.clone(), true).with_metadata(md_f3);
5682            let expected_schema = Arc::new(Schema::new(vec![f1_field, f2_field, f3_field]));
5683            RecordBatch::try_new(
5684                expected_schema,
5685                vec![
5686                    Arc::new(f1_dict) as Arc<dyn Array>,
5687                    Arc::new(f2_dict) as Arc<dyn Array>,
5688                    Arc::new(f3_dict) as Arc<dyn Array>,
5689                ],
5690            )
5691            .unwrap()
5692        }
5693
5694        fn build_expected_fixed() -> RecordBatch {
5695            let f1 =
5696                FixedSizeBinaryArray::try_from_iter(vec![b"abcde", b"12345"].into_iter()).unwrap();
5697            let f2 =
5698                FixedSizeBinaryArray::try_from_iter(vec![b"fghijklmno", b"1234567890"].into_iter())
5699                    .unwrap();
5700            let f3 = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
5701                vec![Some(b"ABCDEF" as &[u8]), None].into_iter(),
5702                6,
5703            )
5704            .unwrap();
5705
5706            // Add Avro named-type metadata for fixed fields
5707            let mut md_f1 = HashMap::new();
5708            md_f1.insert(
5709                crate::schema::AVRO_NAME_METADATA_KEY.to_string(),
5710                "fixed1".to_string(),
5711            );
5712            md_f1.insert(
5713                crate::schema::AVRO_NAMESPACE_METADATA_KEY.to_string(),
5714                "ns1".to_string(),
5715            );
5716
5717            let mut md_f2 = HashMap::new();
5718            md_f2.insert(
5719                crate::schema::AVRO_NAME_METADATA_KEY.to_string(),
5720                "fixed2".to_string(),
5721            );
5722            md_f2.insert(
5723                crate::schema::AVRO_NAMESPACE_METADATA_KEY.to_string(),
5724                "ns2".to_string(),
5725            );
5726
5727            let mut md_f3 = HashMap::new();
5728            md_f3.insert(
5729                crate::schema::AVRO_NAME_METADATA_KEY.to_string(),
5730                "fixed3".to_string(),
5731            );
5732            md_f3.insert(
5733                crate::schema::AVRO_NAMESPACE_METADATA_KEY.to_string(),
5734                "ns1".to_string(),
5735            );
5736
5737            let expected_schema = Arc::new(Schema::new(vec![
5738                Field::new("f1", DataType::FixedSizeBinary(5), false).with_metadata(md_f1),
5739                Field::new("f2", DataType::FixedSizeBinary(10), false).with_metadata(md_f2),
5740                Field::new("f3", DataType::FixedSizeBinary(6), true).with_metadata(md_f3),
5741            ]));
5742
5743            RecordBatch::try_new(
5744                expected_schema,
5745                vec![
5746                    Arc::new(f1) as Arc<dyn Array>,
5747                    Arc::new(f2) as Arc<dyn Array>,
5748                    Arc::new(f3) as Arc<dyn Array>,
5749                ],
5750            )
5751            .unwrap()
5752        }
5753        for (file_name, batch_size, expected, alt_batch_size) in tests {
5754            let file = arrow_test_data(file_name);
5755            let actual = read_file(&file, batch_size, false);
5756            assert_eq!(actual, expected);
5757            let actual2 = read_file(&file, alt_batch_size, false);
5758            assert_eq!(actual2, expected);
5759        }
5760    }
5761
5762    #[test]
5763    #[cfg(feature = "snappy")]
5764    fn test_single_nan() {
5765        let file = arrow_test_data("avro/single_nan.avro");
5766        let actual = read_file(&file, 1, false);
5767        use arrow_array::Float64Array;
5768        let schema = Arc::new(Schema::new(vec![Field::new(
5769            "mycol",
5770            DataType::Float64,
5771            true,
5772        )]));
5773        let col = Float64Array::from(vec![None]);
5774        let expected = RecordBatch::try_new(schema, vec![Arc::new(col)]).unwrap();
5775        assert_eq!(actual, expected);
5776        let actual2 = read_file(&file, 2, false);
5777        assert_eq!(actual2, expected);
5778    }
5779
5780    #[test]
5781    fn test_duration_uuid() {
5782        let batch = read_file("test/data/duration_uuid.avro", 4, false);
5783        let schema = batch.schema();
5784        let fields = schema.fields();
5785        assert_eq!(fields.len(), 2);
5786        assert_eq!(fields[0].name(), "duration_field");
5787        assert_eq!(
5788            fields[0].data_type(),
5789            &DataType::Interval(IntervalUnit::MonthDayNano)
5790        );
5791        assert_eq!(fields[1].name(), "uuid_field");
5792        assert_eq!(fields[1].data_type(), &DataType::FixedSizeBinary(16));
5793        assert_eq!(batch.num_rows(), 4);
5794        assert_eq!(batch.num_columns(), 2);
5795        let duration_array = batch
5796            .column(0)
5797            .as_any()
5798            .downcast_ref::<IntervalMonthDayNanoArray>()
5799            .unwrap();
5800        let expected_duration_array: IntervalMonthDayNanoArray = [
5801            Some(IntervalMonthDayNanoType::make_value(1, 15, 500_000_000)),
5802            Some(IntervalMonthDayNanoType::make_value(0, 5, 2_500_000_000)),
5803            Some(IntervalMonthDayNanoType::make_value(2, 0, 0)),
5804            Some(IntervalMonthDayNanoType::make_value(12, 31, 999_000_000)),
5805        ]
5806        .iter()
5807        .copied()
5808        .collect();
5809        assert_eq!(&expected_duration_array, duration_array);
5810        let uuid_array = batch
5811            .column(1)
5812            .as_any()
5813            .downcast_ref::<FixedSizeBinaryArray>()
5814            .unwrap();
5815        let expected_uuid_array = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
5816            [
5817                Some([
5818                    0xfe, 0x7b, 0xc3, 0x0b, 0x4c, 0xe8, 0x4c, 0x5e, 0xb6, 0x7c, 0x22, 0x34, 0xa2,
5819                    0xd3, 0x8e, 0x66,
5820                ]),
5821                Some([
5822                    0xb3, 0x3f, 0x2a, 0xd7, 0x97, 0xb4, 0x4d, 0xe1, 0x8b, 0xfe, 0x94, 0x94, 0x1d,
5823                    0x60, 0x15, 0x6e,
5824                ]),
5825                Some([
5826                    0x5f, 0x74, 0x92, 0x64, 0x07, 0x4b, 0x40, 0x05, 0x84, 0xbf, 0x11, 0x5e, 0xa8,
5827                    0x4e, 0xd2, 0x0a,
5828                ]),
5829                Some([
5830                    0x08, 0x26, 0xcc, 0x06, 0xd2, 0xe3, 0x45, 0x99, 0xb4, 0xad, 0xaf, 0x5f, 0xa6,
5831                    0x90, 0x5c, 0xdb,
5832                ]),
5833            ]
5834            .into_iter(),
5835            16,
5836        )
5837        .unwrap();
5838        assert_eq!(&expected_uuid_array, uuid_array);
5839    }
5840
5841    #[test]
5842    #[cfg(feature = "snappy")]
5843    fn test_datapage_v2() {
5844        let file = arrow_test_data("avro/datapage_v2.snappy.avro");
5845        let batch = read_file(&file, 8, false);
5846        let a = StringArray::from(vec![
5847            Some("abc"),
5848            Some("abc"),
5849            Some("abc"),
5850            None,
5851            Some("abc"),
5852        ]);
5853        let b = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4), Some(5)]);
5854        let c = Float64Array::from(vec![Some(2.0), Some(3.0), Some(4.0), Some(5.0), Some(2.0)]);
5855        let d = BooleanArray::from(vec![
5856            Some(true),
5857            Some(true),
5858            Some(true),
5859            Some(false),
5860            Some(true),
5861        ]);
5862        let e_values = Int32Array::from(vec![
5863            Some(1),
5864            Some(2),
5865            Some(3),
5866            Some(1),
5867            Some(2),
5868            Some(3),
5869            Some(1),
5870            Some(2),
5871        ]);
5872        let e_offsets = OffsetBuffer::new(ScalarBuffer::from(vec![0i32, 3, 3, 3, 6, 8]));
5873        let e_validity = Some(NullBuffer::from(vec![true, false, false, true, true]));
5874        let field_e = Arc::new(Field::new("item", DataType::Int32, true));
5875        let e = ListArray::new(field_e, e_offsets, Arc::new(e_values), e_validity);
5876        let expected = RecordBatch::try_from_iter_with_nullable([
5877            ("a", Arc::new(a) as Arc<dyn Array>, true),
5878            ("b", Arc::new(b) as Arc<dyn Array>, true),
5879            ("c", Arc::new(c) as Arc<dyn Array>, true),
5880            ("d", Arc::new(d) as Arc<dyn Array>, true),
5881            ("e", Arc::new(e) as Arc<dyn Array>, true),
5882        ])
5883        .unwrap();
5884        assert_eq!(batch, expected);
5885    }
5886
5887    #[test]
5888    fn test_nested_records() {
5889        let f1_f1_1 = StringArray::from(vec!["aaa", "bbb"]);
5890        let f1_f1_2 = Int32Array::from(vec![10, 20]);
5891        let rounded_pi = (std::f64::consts::PI * 100.0).round() / 100.0;
5892        let f1_f1_3_1 = Float64Array::from(vec![rounded_pi, rounded_pi]);
5893        let f1_f1_3 = StructArray::from(vec![(
5894            Arc::new(Field::new("f1_3_1", DataType::Float64, false)),
5895            Arc::new(f1_f1_3_1) as Arc<dyn Array>,
5896        )]);
5897        // Add Avro named-type metadata to nested field f1_3 (ns3.record3)
5898        let mut f1_3_md: HashMap<String, String> = HashMap::new();
5899        f1_3_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns3".to_string());
5900        f1_3_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record3".to_string());
5901        let f1_expected = StructArray::from(vec![
5902            (
5903                Arc::new(Field::new("f1_1", DataType::Utf8, false)),
5904                Arc::new(f1_f1_1) as Arc<dyn Array>,
5905            ),
5906            (
5907                Arc::new(Field::new("f1_2", DataType::Int32, false)),
5908                Arc::new(f1_f1_2) as Arc<dyn Array>,
5909            ),
5910            (
5911                Arc::new(
5912                    Field::new(
5913                        "f1_3",
5914                        DataType::Struct(Fields::from(vec![Field::new(
5915                            "f1_3_1",
5916                            DataType::Float64,
5917                            false,
5918                        )])),
5919                        false,
5920                    )
5921                    .with_metadata(f1_3_md),
5922                ),
5923                Arc::new(f1_f1_3) as Arc<dyn Array>,
5924            ),
5925        ]);
5926        let f2_fields = [
5927            Field::new("f2_1", DataType::Boolean, false),
5928            Field::new("f2_2", DataType::Float32, false),
5929        ];
5930        let f2_struct_builder = StructBuilder::new(
5931            f2_fields
5932                .iter()
5933                .map(|f| Arc::new(f.clone()))
5934                .collect::<Vec<Arc<Field>>>(),
5935            vec![
5936                Box::new(BooleanBuilder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>,
5937                Box::new(Float32Builder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>,
5938            ],
5939        );
5940        let mut f2_list_builder = ListBuilder::new(f2_struct_builder);
5941        {
5942            let struct_builder = f2_list_builder.values();
5943            struct_builder.append(true);
5944            {
5945                let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap();
5946                b.append_value(true);
5947            }
5948            {
5949                let b = struct_builder.field_builder::<Float32Builder>(1).unwrap();
5950                b.append_value(1.2_f32);
5951            }
5952            struct_builder.append(true);
5953            {
5954                let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap();
5955                b.append_value(true);
5956            }
5957            {
5958                let b = struct_builder.field_builder::<Float32Builder>(1).unwrap();
5959                b.append_value(2.2_f32);
5960            }
5961            f2_list_builder.append(true);
5962        }
5963        {
5964            let struct_builder = f2_list_builder.values();
5965            struct_builder.append(true);
5966            {
5967                let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap();
5968                b.append_value(false);
5969            }
5970            {
5971                let b = struct_builder.field_builder::<Float32Builder>(1).unwrap();
5972                b.append_value(10.2_f32);
5973            }
5974            f2_list_builder.append(true);
5975        }
5976
5977        let list_array_with_nullable_items = f2_list_builder.finish();
5978        // Add Avro named-type metadata to f2's list item (ns4.record4)
5979        let mut f2_item_md: HashMap<String, String> = HashMap::new();
5980        f2_item_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record4".to_string());
5981        f2_item_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns4".to_string());
5982        let item_field = Arc::new(
5983            Field::new(
5984                "item",
5985                list_array_with_nullable_items.values().data_type().clone(),
5986                false, // items are non-nullable for f2
5987            )
5988            .with_metadata(f2_item_md),
5989        );
5990        let list_data_type = DataType::List(item_field);
5991        let f2_array_data = list_array_with_nullable_items
5992            .to_data()
5993            .into_builder()
5994            .data_type(list_data_type)
5995            .build()
5996            .unwrap();
5997        let f2_expected = ListArray::from(f2_array_data);
5998        let mut f3_struct_builder = StructBuilder::new(
5999            vec![Arc::new(Field::new("f3_1", DataType::Utf8, false))],
6000            vec![Box::new(StringBuilder::new()) as Box<dyn ArrayBuilder>],
6001        );
6002        f3_struct_builder.append(true);
6003        {
6004            let b = f3_struct_builder.field_builder::<StringBuilder>(0).unwrap();
6005            b.append_value("xyz");
6006        }
6007        f3_struct_builder.append(false);
6008        {
6009            let b = f3_struct_builder.field_builder::<StringBuilder>(0).unwrap();
6010            b.append_null();
6011        }
6012        let f3_expected = f3_struct_builder.finish();
6013        let f4_fields = [Field::new("f4_1", DataType::Int64, false)];
6014        let f4_struct_builder = StructBuilder::new(
6015            f4_fields
6016                .iter()
6017                .map(|f| Arc::new(f.clone()))
6018                .collect::<Vec<Arc<Field>>>(),
6019            vec![Box::new(Int64Builder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>],
6020        );
6021        let mut f4_list_builder = ListBuilder::new(f4_struct_builder);
6022        {
6023            let struct_builder = f4_list_builder.values();
6024            struct_builder.append(true);
6025            {
6026                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6027                b.append_value(200);
6028            }
6029            struct_builder.append(false);
6030            {
6031                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6032                b.append_null();
6033            }
6034            f4_list_builder.append(true);
6035        }
6036        {
6037            let struct_builder = f4_list_builder.values();
6038            struct_builder.append(false);
6039            {
6040                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6041                b.append_null();
6042            }
6043            struct_builder.append(true);
6044            {
6045                let b = struct_builder.field_builder::<Int64Builder>(0).unwrap();
6046                b.append_value(300);
6047            }
6048            f4_list_builder.append(true);
6049        }
6050        let f4_expected = f4_list_builder.finish();
6051        // Add Avro named-type metadata to f4's list item (ns6.record6), item is nullable
6052        let mut f4_item_md: HashMap<String, String> = HashMap::new();
6053        f4_item_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns6".to_string());
6054        f4_item_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record6".to_string());
6055        let f4_item_field = Arc::new(
6056            Field::new("item", f4_expected.values().data_type().clone(), true)
6057                .with_metadata(f4_item_md),
6058        );
6059        let f4_list_data_type = DataType::List(f4_item_field);
6060        let f4_array_data = f4_expected
6061            .to_data()
6062            .into_builder()
6063            .data_type(f4_list_data_type)
6064            .build()
6065            .unwrap();
6066        let f4_expected = ListArray::from(f4_array_data);
6067        // Build Schema with Avro named-type metadata on the top-level f1 and f3 fields
6068        let mut f1_md: HashMap<String, String> = HashMap::new();
6069        f1_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record2".to_string());
6070        f1_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns2".to_string());
6071        let mut f3_md: HashMap<String, String> = HashMap::new();
6072        f3_md.insert(AVRO_NAMESPACE_METADATA_KEY.to_string(), "ns5".to_string());
6073        f3_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "record5".to_string());
6074        let expected_schema = Schema::new(vec![
6075            Field::new("f1", f1_expected.data_type().clone(), false).with_metadata(f1_md),
6076            Field::new("f2", f2_expected.data_type().clone(), false),
6077            Field::new("f3", f3_expected.data_type().clone(), true).with_metadata(f3_md),
6078            Field::new("f4", f4_expected.data_type().clone(), false),
6079        ]);
6080        let expected = RecordBatch::try_new(
6081            Arc::new(expected_schema),
6082            vec![
6083                Arc::new(f1_expected) as Arc<dyn Array>,
6084                Arc::new(f2_expected) as Arc<dyn Array>,
6085                Arc::new(f3_expected) as Arc<dyn Array>,
6086                Arc::new(f4_expected) as Arc<dyn Array>,
6087            ],
6088        )
6089        .unwrap();
6090        let file = arrow_test_data("avro/nested_records.avro");
6091        let batch_large = read_file(&file, 8, false);
6092        assert_eq!(
6093            batch_large, expected,
6094            "Decoded RecordBatch does not match expected data for nested records (batch size 8)"
6095        );
6096        let batch_small = read_file(&file, 3, false);
6097        assert_eq!(
6098            batch_small, expected,
6099            "Decoded RecordBatch does not match expected data for nested records (batch size 3)"
6100        );
6101    }
6102
6103    #[test]
6104    // TODO: avoid requiring snappy for this file
6105    #[cfg(feature = "snappy")]
6106    fn test_repeated_no_annotation() {
6107        use arrow_data::ArrayDataBuilder;
6108        let file = arrow_test_data("avro/repeated_no_annotation.avro");
6109        let batch_large = read_file(&file, 8, false);
6110        // id column
6111        let id_array = Int32Array::from(vec![1, 2, 3, 4, 5, 6]);
6112        // Build the inner Struct<number:int64, kind:utf8>
6113        let number_array = Int64Array::from(vec![
6114            Some(5555555555),
6115            Some(1111111111),
6116            Some(1111111111),
6117            Some(2222222222),
6118            Some(3333333333),
6119        ]);
6120        let kind_array =
6121            StringArray::from(vec![None, Some("home"), Some("home"), None, Some("mobile")]);
6122        let phone_fields = Fields::from(vec![
6123            Field::new("number", DataType::Int64, true),
6124            Field::new("kind", DataType::Utf8, true),
6125        ]);
6126        let phone_struct_data = ArrayDataBuilder::new(DataType::Struct(phone_fields))
6127            .len(5)
6128            .child_data(vec![number_array.into_data(), kind_array.into_data()])
6129            .build()
6130            .unwrap();
6131        let phone_struct_array = StructArray::from(phone_struct_data);
6132        // Build List<item: Struct<...>> with Avro named-type metadata on the *element* field
6133        let phone_list_offsets = Buffer::from_slice_ref([0i32, 0, 0, 0, 1, 2, 5]);
6134        let phone_list_validity = Buffer::from_iter([false, false, true, true, true, true]);
6135        // The Avro schema names this inner record "phone" in namespace "topLevelRecord.phoneNumbers"
6136        let mut phone_item_md = HashMap::new();
6137        phone_item_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "phone".to_string());
6138        phone_item_md.insert(
6139            AVRO_NAMESPACE_METADATA_KEY.to_string(),
6140            "topLevelRecord.phoneNumbers".to_string(),
6141        );
6142        let phone_item_field = Field::new("item", phone_struct_array.data_type().clone(), true)
6143            .with_metadata(phone_item_md);
6144        let phone_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(phone_item_field)))
6145            .len(6)
6146            .add_buffer(phone_list_offsets)
6147            .null_bit_buffer(Some(phone_list_validity))
6148            .child_data(vec![phone_struct_array.into_data()])
6149            .build()
6150            .unwrap();
6151        let phone_list_array = ListArray::from(phone_list_data);
6152        // Wrap in Struct { phone: List<...> }
6153        let phone_numbers_validity = Buffer::from_iter([false, false, true, true, true, true]);
6154        let phone_numbers_field = Field::new("phone", phone_list_array.data_type().clone(), true);
6155        let phone_numbers_struct_data =
6156            ArrayDataBuilder::new(DataType::Struct(Fields::from(vec![phone_numbers_field])))
6157                .len(6)
6158                .null_bit_buffer(Some(phone_numbers_validity))
6159                .child_data(vec![phone_list_array.into_data()])
6160                .build()
6161                .unwrap();
6162        let phone_numbers_struct_array = StructArray::from(phone_numbers_struct_data);
6163        // Build the expected Schema, annotating the top-level "phoneNumbers" field with Avro name/namespace
6164        let mut phone_numbers_md = HashMap::new();
6165        phone_numbers_md.insert(
6166            AVRO_NAME_METADATA_KEY.to_string(),
6167            "phoneNumbers".to_string(),
6168        );
6169        phone_numbers_md.insert(
6170            AVRO_NAMESPACE_METADATA_KEY.to_string(),
6171            "topLevelRecord".to_string(),
6172        );
6173        let id_field = Field::new("id", DataType::Int32, true);
6174        let phone_numbers_schema_field = Field::new(
6175            "phoneNumbers",
6176            phone_numbers_struct_array.data_type().clone(),
6177            true,
6178        )
6179        .with_metadata(phone_numbers_md);
6180        let expected_schema = Schema::new(vec![id_field, phone_numbers_schema_field]);
6181        // Final expected RecordBatch (arrays already carry matching list-element metadata)
6182        let expected = RecordBatch::try_new(
6183            Arc::new(expected_schema),
6184            vec![
6185                Arc::new(id_array) as _,
6186                Arc::new(phone_numbers_struct_array) as _,
6187            ],
6188        )
6189        .unwrap();
6190        assert_eq!(batch_large, expected, "Mismatch for batch_size=8");
6191        let batch_small = read_file(&file, 3, false);
6192        assert_eq!(batch_small, expected, "Mismatch for batch_size=3");
6193    }
6194
6195    #[test]
6196    // TODO: avoid requiring snappy for this file
6197    #[cfg(feature = "snappy")]
6198    fn test_nonnullable_impala() {
6199        let file = arrow_test_data("avro/nonnullable.impala.avro");
6200        let id = Int64Array::from(vec![Some(8)]);
6201        let mut int_array_builder = ListBuilder::new(Int32Builder::new());
6202        {
6203            let vb = int_array_builder.values();
6204            vb.append_value(-1);
6205        }
6206        int_array_builder.append(true); // finalize one sub-list
6207        let int_array = int_array_builder.finish();
6208        let mut iaa_builder = ListBuilder::new(ListBuilder::new(Int32Builder::new()));
6209        {
6210            let inner_list_builder = iaa_builder.values();
6211            {
6212                let vb = inner_list_builder.values();
6213                vb.append_value(-1);
6214                vb.append_value(-2);
6215            }
6216            inner_list_builder.append(true);
6217            inner_list_builder.append(true);
6218        }
6219        iaa_builder.append(true);
6220        let int_array_array = iaa_builder.finish();
6221        let field_names = MapFieldNames {
6222            entry: "entries".to_string(),
6223            key: "key".to_string(),
6224            value: "value".to_string(),
6225        };
6226        let mut int_map_builder =
6227            MapBuilder::new(Some(field_names), StringBuilder::new(), Int32Builder::new());
6228        {
6229            let (keys, vals) = int_map_builder.entries();
6230            keys.append_value("k1");
6231            vals.append_value(-1);
6232        }
6233        int_map_builder.append(true).unwrap(); // finalize map for row 0
6234        let int_map = int_map_builder.finish();
6235        let field_names2 = MapFieldNames {
6236            entry: "entries".to_string(),
6237            key: "key".to_string(),
6238            value: "value".to_string(),
6239        };
6240        let mut ima_builder = ListBuilder::new(MapBuilder::new(
6241            Some(field_names2),
6242            StringBuilder::new(),
6243            Int32Builder::new(),
6244        ));
6245        {
6246            let map_builder = ima_builder.values();
6247            map_builder.append(true).unwrap();
6248            {
6249                let (keys, vals) = map_builder.entries();
6250                keys.append_value("k1");
6251                vals.append_value(1);
6252            }
6253            map_builder.append(true).unwrap();
6254            map_builder.append(true).unwrap();
6255            map_builder.append(true).unwrap();
6256        }
6257        ima_builder.append(true);
6258        let int_map_array_ = ima_builder.finish();
6259        // Helper metadata maps
6260        let meta_nested_struct: HashMap<String, String> = [
6261            ("avro.name", "nested_Struct"),
6262            ("avro.namespace", "topLevelRecord"),
6263        ]
6264        .into_iter()
6265        .map(|(k, v)| (k.to_string(), v.to_string()))
6266        .collect();
6267        let meta_c: HashMap<String, String> = [
6268            ("avro.name", "c"),
6269            ("avro.namespace", "topLevelRecord.nested_Struct"),
6270        ]
6271        .into_iter()
6272        .map(|(k, v)| (k.to_string(), v.to_string()))
6273        .collect();
6274        let meta_d_item_struct: HashMap<String, String> = [
6275            ("avro.name", "D"),
6276            ("avro.namespace", "topLevelRecord.nested_Struct.c"),
6277        ]
6278        .into_iter()
6279        .map(|(k, v)| (k.to_string(), v.to_string()))
6280        .collect();
6281        let meta_g_value: HashMap<String, String> = [
6282            ("avro.name", "G"),
6283            ("avro.namespace", "topLevelRecord.nested_Struct"),
6284        ]
6285        .into_iter()
6286        .map(|(k, v)| (k.to_string(), v.to_string()))
6287        .collect();
6288        let meta_h: HashMap<String, String> = [
6289            ("avro.name", "h"),
6290            ("avro.namespace", "topLevelRecord.nested_Struct.G"),
6291        ]
6292        .into_iter()
6293        .map(|(k, v)| (k.to_string(), v.to_string()))
6294        .collect();
6295        // Types used multiple times below
6296        let ef_struct_field = Arc::new(
6297            Field::new(
6298                "item",
6299                DataType::Struct(
6300                    vec![
6301                        Field::new("e", DataType::Int32, true),
6302                        Field::new("f", DataType::Utf8, true),
6303                    ]
6304                    .into(),
6305                ),
6306                true,
6307            )
6308            .with_metadata(meta_d_item_struct.clone()),
6309        );
6310        let d_inner_list_field = Arc::new(Field::new(
6311            "item",
6312            DataType::List(ef_struct_field.clone()),
6313            true,
6314        ));
6315        let d_field = Field::new("D", DataType::List(d_inner_list_field.clone()), true);
6316        // G.value.h.i : List<Float64>
6317        let i_list_field = Arc::new(Field::new("item", DataType::Float64, true));
6318        let i_field = Field::new("i", DataType::List(i_list_field.clone()), true);
6319        // G.value.h : Struct<{ i: List<Float64> }> with metadata (h)
6320        let h_field = Field::new("h", DataType::Struct(vec![i_field.clone()].into()), true)
6321            .with_metadata(meta_h.clone());
6322        // G.value : Struct<{ h: ... }> with metadata (G)
6323        let g_value_struct_field = Field::new(
6324            "value",
6325            DataType::Struct(vec![h_field.clone()].into()),
6326            true,
6327        )
6328        .with_metadata(meta_g_value.clone());
6329        // entries struct for Map G
6330        let entries_struct_field = Field::new(
6331            "entries",
6332            DataType::Struct(
6333                vec![
6334                    Field::new("key", DataType::Utf8, false),
6335                    g_value_struct_field.clone(),
6336                ]
6337                .into(),
6338            ),
6339            false,
6340        );
6341        // Top-level nested_Struct fields (include metadata on "c")
6342        let a_field = Arc::new(Field::new("a", DataType::Int32, true));
6343        let b_field = Arc::new(Field::new(
6344            "B",
6345            DataType::List(Arc::new(Field::new("item", DataType::Int32, true))),
6346            true,
6347        ));
6348        let c_field = Arc::new(
6349            Field::new("c", DataType::Struct(vec![d_field.clone()].into()), true)
6350                .with_metadata(meta_c.clone()),
6351        );
6352        let g_field = Arc::new(Field::new(
6353            "G",
6354            DataType::Map(Arc::new(entries_struct_field.clone()), false),
6355            true,
6356        ));
6357        // Now create builders that match these exact field types (so nested types carry metadata)
6358        let mut nested_sb = StructBuilder::new(
6359            vec![
6360                a_field.clone(),
6361                b_field.clone(),
6362                c_field.clone(),
6363                g_field.clone(),
6364            ],
6365            vec![
6366                Box::new(Int32Builder::new()),
6367                Box::new(ListBuilder::new(Int32Builder::new())),
6368                {
6369                    // builder for "c" with correctly typed "D" including metadata on inner list item
6370                    Box::new(StructBuilder::new(
6371                        vec![Arc::new(d_field.clone())],
6372                        vec![Box::new({
6373                            let ef_struct_builder = StructBuilder::new(
6374                                vec![
6375                                    Arc::new(Field::new("e", DataType::Int32, true)),
6376                                    Arc::new(Field::new("f", DataType::Utf8, true)),
6377                                ],
6378                                vec![
6379                                    Box::new(Int32Builder::new()),
6380                                    Box::new(StringBuilder::new()),
6381                                ],
6382                            );
6383                            // Inner list that holds Struct<e,f> with Avro named-type metadata ("D")
6384                            let list_of_ef = ListBuilder::new(ef_struct_builder)
6385                                .with_field(ef_struct_field.clone());
6386                            // Outer list for "D"
6387                            ListBuilder::new(list_of_ef)
6388                        })],
6389                    ))
6390                },
6391                {
6392                    let map_field_names = MapFieldNames {
6393                        entry: "entries".to_string(),
6394                        key: "key".to_string(),
6395                        value: "value".to_string(),
6396                    };
6397                    let i_list_builder = ListBuilder::new(Float64Builder::new());
6398                    let h_struct_builder = StructBuilder::new(
6399                        vec![Arc::new(Field::new(
6400                            "i",
6401                            DataType::List(i_list_field.clone()),
6402                            true,
6403                        ))],
6404                        vec![Box::new(i_list_builder)],
6405                    );
6406                    let g_value_builder = StructBuilder::new(
6407                        vec![Arc::new(
6408                            Field::new("h", DataType::Struct(vec![i_field.clone()].into()), true)
6409                                .with_metadata(meta_h.clone()),
6410                        )],
6411                        vec![Box::new(h_struct_builder)],
6412                    );
6413                    // Use with_values_field to attach metadata to "value" field in the map's entries
6414                    let map_builder = MapBuilder::new(
6415                        Some(map_field_names),
6416                        StringBuilder::new(),
6417                        g_value_builder,
6418                    )
6419                    .with_values_field(Arc::new(
6420                        Field::new(
6421                            "value",
6422                            DataType::Struct(vec![h_field.clone()].into()),
6423                            true,
6424                        )
6425                        .with_metadata(meta_g_value.clone()),
6426                    ));
6427
6428                    Box::new(map_builder)
6429                },
6430            ],
6431        );
6432        nested_sb.append(true);
6433        {
6434            let a_builder = nested_sb.field_builder::<Int32Builder>(0).unwrap();
6435            a_builder.append_value(-1);
6436        }
6437        {
6438            let b_builder = nested_sb
6439                .field_builder::<ListBuilder<Int32Builder>>(1)
6440                .unwrap();
6441            {
6442                let vb = b_builder.values();
6443                vb.append_value(-1);
6444            }
6445            b_builder.append(true);
6446        }
6447        {
6448            let c_struct_builder = nested_sb.field_builder::<StructBuilder>(2).unwrap();
6449            c_struct_builder.append(true);
6450            let d_list_builder = c_struct_builder
6451                .field_builder::<ListBuilder<ListBuilder<StructBuilder>>>(0)
6452                .unwrap();
6453            {
6454                let sub_list_builder = d_list_builder.values();
6455                {
6456                    let ef_struct = sub_list_builder.values();
6457                    ef_struct.append(true);
6458                    {
6459                        let e_b = ef_struct.field_builder::<Int32Builder>(0).unwrap();
6460                        e_b.append_value(-1);
6461                        let f_b = ef_struct.field_builder::<StringBuilder>(1).unwrap();
6462                        f_b.append_value("nonnullable");
6463                    }
6464                    sub_list_builder.append(true);
6465                }
6466                d_list_builder.append(true);
6467            }
6468        }
6469        {
6470            let g_map_builder = nested_sb
6471                .field_builder::<MapBuilder<StringBuilder, StructBuilder>>(3)
6472                .unwrap();
6473            g_map_builder.append(true).unwrap();
6474        }
6475        let nested_struct = nested_sb.finish();
6476        let schema = Arc::new(arrow_schema::Schema::new(vec![
6477            Field::new("ID", id.data_type().clone(), true),
6478            Field::new("Int_Array", int_array.data_type().clone(), true),
6479            Field::new("int_array_array", int_array_array.data_type().clone(), true),
6480            Field::new("Int_Map", int_map.data_type().clone(), true),
6481            Field::new("int_map_array", int_map_array_.data_type().clone(), true),
6482            Field::new("nested_Struct", nested_struct.data_type().clone(), true)
6483                .with_metadata(meta_nested_struct.clone()),
6484        ]));
6485        let expected = RecordBatch::try_new(
6486            schema,
6487            vec![
6488                Arc::new(id) as Arc<dyn Array>,
6489                Arc::new(int_array),
6490                Arc::new(int_array_array),
6491                Arc::new(int_map),
6492                Arc::new(int_map_array_),
6493                Arc::new(nested_struct),
6494            ],
6495        )
6496        .unwrap();
6497        let batch_large = read_file(&file, 8, false);
6498        assert_eq!(batch_large, expected, "Mismatch for batch_size=8");
6499        let batch_small = read_file(&file, 3, false);
6500        assert_eq!(batch_small, expected, "Mismatch for batch_size=3");
6501    }
6502
6503    #[test]
6504    fn test_nonnullable_impala_strict() {
6505        let file = arrow_test_data("avro/nonnullable.impala.avro");
6506        let err = read_file_strict(&file, 8, false).unwrap_err();
6507        assert!(err.to_string().contains(
6508            "Found Avro union of the form ['T','null'], which is disallowed in strict_mode"
6509        ));
6510    }
6511
6512    #[test]
6513    // TODO: avoid requiring snappy for this file
6514    #[cfg(feature = "snappy")]
6515    fn test_nullable_impala() {
6516        let file = arrow_test_data("avro/nullable.impala.avro");
6517        let batch1 = read_file(&file, 3, false);
6518        let batch2 = read_file(&file, 8, false);
6519        assert_eq!(batch1, batch2);
6520        let batch = batch1;
6521        assert_eq!(batch.num_rows(), 7);
6522        let id_array = batch
6523            .column(0)
6524            .as_any()
6525            .downcast_ref::<Int64Array>()
6526            .expect("id column should be an Int64Array");
6527        let expected_ids = [1, 2, 3, 4, 5, 6, 7];
6528        for (i, &expected_id) in expected_ids.iter().enumerate() {
6529            assert_eq!(id_array.value(i), expected_id, "Mismatch in id at row {i}",);
6530        }
6531        let int_array = batch
6532            .column(1)
6533            .as_any()
6534            .downcast_ref::<ListArray>()
6535            .expect("int_array column should be a ListArray");
6536        {
6537            let offsets = int_array.value_offsets();
6538            let start = offsets[0] as usize;
6539            let end = offsets[1] as usize;
6540            let values = int_array
6541                .values()
6542                .as_any()
6543                .downcast_ref::<Int32Array>()
6544                .expect("Values of int_array should be an Int32Array");
6545            let row0: Vec<Option<i32>> = (start..end).map(|i| Some(values.value(i))).collect();
6546            assert_eq!(
6547                row0,
6548                vec![Some(1), Some(2), Some(3)],
6549                "Mismatch in int_array row 0"
6550            );
6551        }
6552        let nested_struct = batch
6553            .column(5)
6554            .as_any()
6555            .downcast_ref::<StructArray>()
6556            .expect("nested_struct column should be a StructArray");
6557        let a_array = nested_struct
6558            .column_by_name("A")
6559            .expect("Field A should exist in nested_struct")
6560            .as_any()
6561            .downcast_ref::<Int32Array>()
6562            .expect("Field A should be an Int32Array");
6563        assert_eq!(a_array.value(0), 1, "Mismatch in nested_struct.A at row 0");
6564        assert!(
6565            !a_array.is_valid(1),
6566            "Expected null in nested_struct.A at row 1"
6567        );
6568        assert!(
6569            !a_array.is_valid(3),
6570            "Expected null in nested_struct.A at row 3"
6571        );
6572        assert_eq!(a_array.value(6), 7, "Mismatch in nested_struct.A at row 6");
6573    }
6574
6575    #[test]
6576    fn test_nullable_impala_strict() {
6577        let file = arrow_test_data("avro/nullable.impala.avro");
6578        let err = read_file_strict(&file, 8, false).unwrap_err();
6579        assert!(err.to_string().contains(
6580            "Found Avro union of the form ['T','null'], which is disallowed in strict_mode"
6581        ));
6582    }
6583
6584    #[test]
6585    fn test_nested_record_type_reuse() {
6586        // The .avro file has the following schema:
6587        // {
6588        // "type" : "record",
6589        // "name" : "Record",
6590        // "fields" : [ {
6591        //     "name" : "nested",
6592        //     "type" : {
6593        //     "type" : "record",
6594        //     "name" : "Nested",
6595        //     "fields" : [ {
6596        //         "name" : "nested_int",
6597        //         "type" : "int"
6598        //     } ]
6599        //     }
6600        // }, {
6601        //     "name" : "nestedRecord",
6602        //     "type" : "Nested"
6603        // }, {
6604        //     "name" : "nestedArray",
6605        //     "type" : {
6606        //     "type" : "array",
6607        //     "items" : "Nested"
6608        //     }
6609        // } ]
6610        // }
6611        let batch = read_file("test/data/nested_record_reuse.avro", 8, false);
6612        let schema = batch.schema();
6613
6614        // Verify schema structure
6615        assert_eq!(schema.fields().len(), 3);
6616        let fields = schema.fields();
6617        assert_eq!(fields[0].name(), "nested");
6618        assert_eq!(fields[1].name(), "nestedRecord");
6619        assert_eq!(fields[2].name(), "nestedArray");
6620        assert!(matches!(fields[0].data_type(), DataType::Struct(_)));
6621        assert!(matches!(fields[1].data_type(), DataType::Struct(_)));
6622        assert!(matches!(fields[2].data_type(), DataType::List(_)));
6623
6624        // Validate that the nested record type
6625        if let DataType::Struct(nested_fields) = fields[0].data_type() {
6626            assert_eq!(nested_fields.len(), 1);
6627            assert_eq!(nested_fields[0].name(), "nested_int");
6628            assert_eq!(nested_fields[0].data_type(), &DataType::Int32);
6629        }
6630
6631        // Validate that the nested record type is reused
6632        assert_eq!(fields[0].data_type(), fields[1].data_type());
6633        if let DataType::List(array_field) = fields[2].data_type() {
6634            assert_eq!(array_field.data_type(), fields[0].data_type());
6635        }
6636
6637        // Validate data
6638        assert_eq!(batch.num_rows(), 2);
6639        assert_eq!(batch.num_columns(), 3);
6640
6641        // Validate the first column (nested)
6642        let nested_col = batch
6643            .column(0)
6644            .as_any()
6645            .downcast_ref::<StructArray>()
6646            .unwrap();
6647        let nested_int_array = nested_col
6648            .column_by_name("nested_int")
6649            .unwrap()
6650            .as_any()
6651            .downcast_ref::<Int32Array>()
6652            .unwrap();
6653        assert_eq!(nested_int_array.value(0), 42);
6654        assert_eq!(nested_int_array.value(1), 99);
6655
6656        // Validate the second column (nestedRecord)
6657        let nested_record_col = batch
6658            .column(1)
6659            .as_any()
6660            .downcast_ref::<StructArray>()
6661            .unwrap();
6662        let nested_record_int_array = nested_record_col
6663            .column_by_name("nested_int")
6664            .unwrap()
6665            .as_any()
6666            .downcast_ref::<Int32Array>()
6667            .unwrap();
6668        assert_eq!(nested_record_int_array.value(0), 100);
6669        assert_eq!(nested_record_int_array.value(1), 200);
6670
6671        // Validate the third column (nestedArray)
6672        let nested_array_col = batch
6673            .column(2)
6674            .as_any()
6675            .downcast_ref::<ListArray>()
6676            .unwrap();
6677        assert_eq!(nested_array_col.len(), 2);
6678        let first_array_struct = nested_array_col.value(0);
6679        let first_array_struct_array = first_array_struct
6680            .as_any()
6681            .downcast_ref::<StructArray>()
6682            .unwrap();
6683        let first_array_int_values = first_array_struct_array
6684            .column_by_name("nested_int")
6685            .unwrap()
6686            .as_any()
6687            .downcast_ref::<Int32Array>()
6688            .unwrap();
6689        assert_eq!(first_array_int_values.len(), 3);
6690        assert_eq!(first_array_int_values.value(0), 1);
6691        assert_eq!(first_array_int_values.value(1), 2);
6692        assert_eq!(first_array_int_values.value(2), 3);
6693    }
6694
6695    #[test]
6696    fn test_enum_type_reuse() {
6697        // The .avro file has the following schema:
6698        // {
6699        //     "type" : "record",
6700        //     "name" : "Record",
6701        //     "fields" : [ {
6702        //       "name" : "status",
6703        //       "type" : {
6704        //         "type" : "enum",
6705        //         "name" : "Status",
6706        //         "symbols" : [ "ACTIVE", "INACTIVE", "PENDING" ]
6707        //       }
6708        //     }, {
6709        //       "name" : "backupStatus",
6710        //       "type" : "Status"
6711        //     }, {
6712        //       "name" : "statusHistory",
6713        //       "type" : {
6714        //         "type" : "array",
6715        //         "items" : "Status"
6716        //       }
6717        //     } ]
6718        //   }
6719        let batch = read_file("test/data/enum_reuse.avro", 8, false);
6720        let schema = batch.schema();
6721
6722        // Verify schema structure
6723        assert_eq!(schema.fields().len(), 3);
6724        let fields = schema.fields();
6725        assert_eq!(fields[0].name(), "status");
6726        assert_eq!(fields[1].name(), "backupStatus");
6727        assert_eq!(fields[2].name(), "statusHistory");
6728        assert!(matches!(fields[0].data_type(), DataType::Dictionary(_, _)));
6729        assert!(matches!(fields[1].data_type(), DataType::Dictionary(_, _)));
6730        assert!(matches!(fields[2].data_type(), DataType::List(_)));
6731
6732        if let DataType::Dictionary(key_type, value_type) = fields[0].data_type() {
6733            assert_eq!(key_type.as_ref(), &DataType::Int32);
6734            assert_eq!(value_type.as_ref(), &DataType::Utf8);
6735        }
6736
6737        // Validate that the enum types are reused
6738        assert_eq!(fields[0].data_type(), fields[1].data_type());
6739        if let DataType::List(array_field) = fields[2].data_type() {
6740            assert_eq!(array_field.data_type(), fields[0].data_type());
6741        }
6742
6743        // Validate data - should have 2 rows
6744        assert_eq!(batch.num_rows(), 2);
6745        assert_eq!(batch.num_columns(), 3);
6746
6747        // Get status enum values
6748        let status_col = batch
6749            .column(0)
6750            .as_any()
6751            .downcast_ref::<DictionaryArray<Int32Type>>()
6752            .unwrap();
6753        let status_values = status_col
6754            .values()
6755            .as_any()
6756            .downcast_ref::<StringArray>()
6757            .unwrap();
6758
6759        // First row should be "ACTIVE", second row should be "PENDING"
6760        assert_eq!(
6761            status_values.value(status_col.key(0).unwrap() as usize),
6762            "ACTIVE"
6763        );
6764        assert_eq!(
6765            status_values.value(status_col.key(1).unwrap() as usize),
6766            "PENDING"
6767        );
6768
6769        // Get backupStatus enum values (same as status)
6770        let backup_status_col = batch
6771            .column(1)
6772            .as_any()
6773            .downcast_ref::<DictionaryArray<Int32Type>>()
6774            .unwrap();
6775        let backup_status_values = backup_status_col
6776            .values()
6777            .as_any()
6778            .downcast_ref::<StringArray>()
6779            .unwrap();
6780
6781        // First row should be "INACTIVE", second row should be "ACTIVE"
6782        assert_eq!(
6783            backup_status_values.value(backup_status_col.key(0).unwrap() as usize),
6784            "INACTIVE"
6785        );
6786        assert_eq!(
6787            backup_status_values.value(backup_status_col.key(1).unwrap() as usize),
6788            "ACTIVE"
6789        );
6790
6791        // Get statusHistory array
6792        let status_history_col = batch
6793            .column(2)
6794            .as_any()
6795            .downcast_ref::<ListArray>()
6796            .unwrap();
6797        assert_eq!(status_history_col.len(), 2);
6798
6799        // Validate first row's array data
6800        let first_array_dict = status_history_col.value(0);
6801        let first_array_dict_array = first_array_dict
6802            .as_any()
6803            .downcast_ref::<DictionaryArray<Int32Type>>()
6804            .unwrap();
6805        let first_array_values = first_array_dict_array
6806            .values()
6807            .as_any()
6808            .downcast_ref::<StringArray>()
6809            .unwrap();
6810
6811        // First row: ["PENDING", "ACTIVE", "INACTIVE"]
6812        assert_eq!(first_array_dict_array.len(), 3);
6813        assert_eq!(
6814            first_array_values.value(first_array_dict_array.key(0).unwrap() as usize),
6815            "PENDING"
6816        );
6817        assert_eq!(
6818            first_array_values.value(first_array_dict_array.key(1).unwrap() as usize),
6819            "ACTIVE"
6820        );
6821        assert_eq!(
6822            first_array_values.value(first_array_dict_array.key(2).unwrap() as usize),
6823            "INACTIVE"
6824        );
6825    }
6826
6827    #[test]
6828    fn test_bad_varint_bug_nullable_array_items() {
6829        use flate2::read::GzDecoder;
6830        use std::io::Read;
6831        let manifest_dir = env!("CARGO_MANIFEST_DIR");
6832        let gz_path = format!("{manifest_dir}/test/data/bad-varint-bug.avro.gz");
6833        let gz_file = File::open(&gz_path).expect("test file should exist");
6834        let mut decoder = GzDecoder::new(gz_file);
6835        let mut avro_bytes = Vec::new();
6836        decoder
6837            .read_to_end(&mut avro_bytes)
6838            .expect("should decompress");
6839        let reader_arrow_schema = Schema::new(vec![Field::new(
6840            "int_array",
6841            DataType::List(Arc::new(Field::new("element", DataType::Int32, true))),
6842            true,
6843        )])
6844        .with_metadata(HashMap::from([("avro.name".into(), "table".into())]));
6845        let reader_schema = AvroSchema::try_from(&reader_arrow_schema)
6846            .expect("should convert Arrow schema to Avro");
6847        let mut reader = ReaderBuilder::new()
6848            .with_reader_schema(reader_schema)
6849            .build(Cursor::new(avro_bytes))
6850            .expect("should build reader");
6851        let batch = reader
6852            .next()
6853            .expect("should have one batch")
6854            .expect("reading should succeed without bad varint error");
6855        assert_eq!(batch.num_rows(), 1);
6856        let list_col = batch
6857            .column(0)
6858            .as_any()
6859            .downcast_ref::<ListArray>()
6860            .expect("should be ListArray");
6861        assert_eq!(list_col.len(), 1);
6862        let values = list_col.values();
6863        let int_values = values.as_primitive::<Int32Type>();
6864        assert_eq!(int_values.len(), 2);
6865        assert_eq!(int_values.value(0), 1);
6866        assert_eq!(int_values.value(1), 2);
6867    }
6868
6869    #[test]
6870    fn test_nested_record_field_addition() {
6871        let file = arrow_test_data("avro/nested_records.avro");
6872
6873        // Adds fields to the writer schema:
6874        // * "ns2.record2" / "f1_4"
6875        //   - nullable
6876        //   - added last
6877        //   - the containing "f1" field is made nullable in the reader
6878        // * "ns4.record4" / "f2_3"
6879        //   - non-nullable with an integer default value
6880        //   - resolution of a record nested in an array
6881        // * "ns5.record5" / "f3_0"
6882        //   - non-nullable with a string default value
6883        //   - prepended before existing fields in the schema order
6884        let reader_schema = AvroSchema::new(
6885            r#"
6886            {
6887                "type": "record",
6888                "name": "record1",
6889                "namespace": "ns1",
6890                "fields": [
6891                    {
6892                        "name": "f1",
6893                        "type": [
6894                            "null",
6895                            {
6896                                "type": "record",
6897                                "name": "record2",
6898                                "namespace": "ns2",
6899                                "fields": [
6900                                    {
6901                                        "name": "f1_1",
6902                                        "type": "string"
6903                                    },
6904                                    {
6905                                        "name": "f1_2",
6906                                        "type": "int"
6907                                    },
6908                                    {
6909                                        "name": "f1_3",
6910                                        "type": {
6911                                            "type": "record",
6912                                            "name": "record3",
6913                                            "namespace": "ns3",
6914                                            "fields": [
6915                                                {
6916                                                    "name": "f1_3_1",
6917                                                    "type": "double"
6918                                                }
6919                                            ]
6920                                        }
6921                                    },
6922                                    {
6923                                        "name": "f1_4",
6924                                        "type": ["null", "int"],
6925                                        "default": null
6926                                    }
6927                                ]
6928                            }
6929                        ]
6930                    },
6931                    {
6932                        "name": "f2",
6933                        "type": {
6934                            "type": "array",
6935                            "items": {
6936                                "type": "record",
6937                                "name": "record4",
6938                                "namespace": "ns4",
6939                                "fields": [
6940                                    {
6941                                        "name": "f2_1",
6942                                        "type": "boolean"
6943                                    },
6944                                    {
6945                                        "name": "f2_2",
6946                                        "type": "float"
6947                                    },
6948                                    {
6949                                        "name": "f2_3",
6950                                        "type": ["null", "int"],
6951                                        "default": 42
6952                                    }
6953                                ]
6954                            }
6955                        }
6956                    },
6957                    {
6958                        "name": "f3",
6959                        "type": [
6960                            "null",
6961                            {
6962                                "type": "record",
6963                                "name": "record5",
6964                                "namespace": "ns5",
6965                                "fields": [
6966                                    {
6967                                        "name": "f3_0",
6968                                        "type": "string",
6969                                        "default": "lorem ipsum"
6970                                    },
6971                                    {
6972                                        "name": "f3_1",
6973                                        "type": "string"
6974                                    }
6975                                ]
6976                            }
6977                        ],
6978                        "default": null
6979                    },
6980                    {
6981                        "name": "f4",
6982                        "type": {
6983                            "type": "array",
6984                            "items": [
6985                                "null",
6986                                {
6987                                    "type": "record",
6988                                    "name": "record6",
6989                                    "namespace": "ns6",
6990                                    "fields": [
6991                                        {
6992                                            "name": "f4_1",
6993                                            "type": "long"
6994                                        }
6995                                    ]
6996                                }
6997                            ]
6998                        }
6999                    }
7000                ]
7001            }
7002            "#
7003            .to_string(),
7004        );
7005
7006        let file = File::open(&file).unwrap();
7007        let mut reader = ReaderBuilder::new()
7008            .with_reader_schema(reader_schema)
7009            .build(BufReader::new(file))
7010            .expect("reader with evolved reader schema should be built successfully");
7011
7012        let batch = reader
7013            .next()
7014            .expect("should have at least one batch")
7015            .expect("reading should succeed");
7016
7017        assert!(batch.num_rows() > 0);
7018
7019        let schema = batch.schema();
7020
7021        let f1_field = schema.field_with_name("f1").expect("f1 field should exist");
7022        if let DataType::Struct(f1_fields) = f1_field.data_type() {
7023            let (_, f1_4) = f1_fields
7024                .find("f1_4")
7025                .expect("f1_4 field should be present in record2");
7026            assert!(f1_4.is_nullable(), "f1_4 should be nullable");
7027            assert_eq!(f1_4.data_type(), &DataType::Int32, "f1_4 should be Int32");
7028            assert_eq!(
7029                f1_4.metadata().get("avro.field.default"),
7030                Some(&"null".to_string()),
7031                "f1_4 should have null default value in metadata"
7032            );
7033        } else {
7034            panic!("f1 should be a struct");
7035        }
7036
7037        let f2_field = schema.field_with_name("f2").expect("f2 field should exist");
7038        if let DataType::List(f2_items_field) = f2_field.data_type() {
7039            if let DataType::Struct(f2_items_fields) = f2_items_field.data_type() {
7040                let (_, f2_3) = f2_items_fields
7041                    .find("f2_3")
7042                    .expect("f2_3 field should be present in record4");
7043                assert!(f2_3.is_nullable(), "f2_3 should be nullable");
7044                assert_eq!(f2_3.data_type(), &DataType::Int32, "f2_3 should be Int32");
7045                assert_eq!(
7046                    f2_3.metadata().get("avro.field.default"),
7047                    Some(&"42".to_string()),
7048                    "f2_3 should have 42 default value in metadata"
7049                );
7050            } else {
7051                panic!("f2 array items should be a struct");
7052            }
7053        } else {
7054            panic!("f2 should be a list");
7055        }
7056
7057        let f3_field = schema.field_with_name("f3").expect("f3 field should exist");
7058        assert!(f3_field.is_nullable(), "f3 should be nullable");
7059        if let DataType::Struct(f3_fields) = f3_field.data_type() {
7060            let (_, f3_0) = f3_fields
7061                .find("f3_0")
7062                .expect("f3_0 field should be present in record5");
7063            assert!(!f3_0.is_nullable(), "f3_0 should be non-nullable");
7064            assert_eq!(f3_0.data_type(), &DataType::Utf8, "f3_0 should be a string");
7065            assert_eq!(
7066                f3_0.metadata().get("avro.field.default"),
7067                Some(&"\"lorem ipsum\"".to_string()),
7068                "f3_0 should have \"lorem ipsum\" default value in metadata"
7069            );
7070        } else {
7071            panic!("f3 should be a struct");
7072        }
7073
7074        // Verify the actual values in the columns match the expected defaults
7075        let num_rows = batch.num_rows();
7076
7077        // Check f1_4 values (should all be null since default is null)
7078        let f1_array = batch
7079            .column_by_name("f1")
7080            .expect("f1 column should exist")
7081            .as_struct();
7082        let f1_4_array = f1_array
7083            .column_by_name("f1_4")
7084            .expect("f1_4 column should exist in f1 struct")
7085            .as_primitive::<Int32Type>();
7086
7087        assert_eq!(f1_4_array.null_count(), num_rows);
7088
7089        let f2_array = batch
7090            .column_by_name("f2")
7091            .expect("f2 column should exist")
7092            .as_list::<i32>();
7093
7094        for i in 0..num_rows {
7095            assert!(!f2_array.is_null(i));
7096            let f2_value = f2_array.value(i);
7097            let f2_record_array = f2_value.as_struct();
7098            let f2_3_array = f2_record_array
7099                .column_by_name("f2_3")
7100                .expect("f2_3 column should exist in f2 array items")
7101                .as_primitive::<Int32Type>();
7102
7103            for j in 0..f2_3_array.len() {
7104                assert!(!f2_3_array.is_null(j));
7105                assert_eq!(f2_3_array.value(j), 42);
7106            }
7107        }
7108
7109        let f3_array = batch
7110            .column_by_name("f3")
7111            .expect("f3 column should exist")
7112            .as_struct();
7113        let f3_0_array = f3_array
7114            .column_by_name("f3_0")
7115            .expect("f3_0 column should exist in f3 struct")
7116            .as_string::<i32>();
7117
7118        for i in 0..num_rows {
7119            // Only check f3_0 when the parent f3 struct is not null
7120            if !f3_array.is_null(i) {
7121                assert!(!f3_0_array.is_null(i));
7122                assert_eq!(f3_0_array.value(i), "lorem ipsum");
7123            }
7124        }
7125    }
7126
7127    fn corrupt_first_block_payload_byte(
7128        mut bytes: Vec<u8>,
7129        field_offset: usize,
7130        expected_original: u8,
7131        replacement: u8,
7132    ) -> Vec<u8> {
7133        let mut header_decoder = HeaderDecoder::default();
7134        let header_len = header_decoder.decode(&bytes).expect("decode header");
7135        assert!(header_decoder.flush().is_some(), "decode complete header");
7136
7137        let mut cursor = &bytes[header_len..];
7138        let (_, count_len) = crate::reader::vlq::read_varint(cursor).expect("decode block count");
7139        cursor = &cursor[count_len..];
7140        let (_, size_len) = crate::reader::vlq::read_varint(cursor).expect("decode block size");
7141        let data_start = header_len + count_len + size_len;
7142        let target = data_start + field_offset;
7143
7144        assert!(
7145            target < bytes.len(),
7146            "target byte offset {target} out of bounds for input length {}",
7147            bytes.len()
7148        );
7149        assert_eq!(
7150            bytes[target], expected_original,
7151            "unexpected original byte at payload offset {field_offset}"
7152        );
7153        bytes[target] = replacement;
7154        bytes
7155    }
7156
7157    #[test]
7158    fn ocf_projection_rejects_overflowing_varint_in_skipped_long_field() {
7159        // Writer row payload is [bad_long=i64::MIN][keep=7]. The first field is encoded as
7160        // 10-byte VLQ ending in 0x01. Flipping that terminator to 0x02 creates an overflow
7161        // varint that must fail.
7162        let writer_schema = Schema::new(vec![
7163            Field::new("bad_long", DataType::Int64, false),
7164            Field::new("keep", DataType::Int32, false),
7165        ]);
7166        let batch = RecordBatch::try_new(
7167            Arc::new(writer_schema.clone()),
7168            vec![
7169                Arc::new(Int64Array::from(vec![i64::MIN])) as ArrayRef,
7170                Arc::new(Int32Array::from(vec![7])) as ArrayRef,
7171            ],
7172        )
7173        .expect("build writer batch");
7174        let bytes = write_ocf(&writer_schema, &[batch]);
7175        let mutated = corrupt_first_block_payload_byte(bytes, 9, 0x01, 0x02);
7176
7177        let err = ReaderBuilder::new()
7178            .build(Cursor::new(mutated.clone()))
7179            .expect("build full reader")
7180            .collect::<Result<Vec<_>, _>>()
7181            .expect_err("full decode should reject malformed varint");
7182        assert!(matches!(err, ArrowError::AvroError(_)));
7183        assert!(err.to_string().contains("bad varint"));
7184
7185        let err = ReaderBuilder::new()
7186            .with_projection(vec![1])
7187            .build(Cursor::new(mutated))
7188            .expect("build projected reader")
7189            .collect::<Result<Vec<_>, _>>()
7190            .expect_err("projection must also reject malformed skipped varint");
7191        assert!(matches!(err, ArrowError::AvroError(_)));
7192        assert!(err.to_string().contains("bad varint"));
7193    }
7194
7195    #[test]
7196    fn ocf_projection_rejects_i32_overflow_in_skipped_int_field() {
7197        // Writer row payload is [bad_int=i32::MIN][keep=11]. The first field encodes to
7198        // ff ff ff ff 0f. Flipping 0x0f -> 0x10 keeps a syntactically valid varint, but now
7199        // its value exceeds u32::MAX and must fail Int32 validation even when projected out.
7200        let writer_schema = Schema::new(vec![
7201            Field::new("bad_int", DataType::Int32, false),
7202            Field::new("keep", DataType::Int64, false),
7203        ]);
7204        let batch = RecordBatch::try_new(
7205            Arc::new(writer_schema.clone()),
7206            vec![
7207                Arc::new(Int32Array::from(vec![i32::MIN])) as ArrayRef,
7208                Arc::new(Int64Array::from(vec![11])) as ArrayRef,
7209            ],
7210        )
7211        .expect("build writer batch");
7212        let bytes = write_ocf(&writer_schema, &[batch]);
7213        let mutated = corrupt_first_block_payload_byte(bytes, 4, 0x0f, 0x10);
7214
7215        let err = ReaderBuilder::new()
7216            .build(Cursor::new(mutated.clone()))
7217            .expect("build full reader")
7218            .collect::<Result<Vec<_>, _>>()
7219            .expect_err("full decode should reject int overflow");
7220        assert!(matches!(err, ArrowError::AvroError(_)));
7221        assert!(err.to_string().contains("varint overflow"));
7222
7223        let err = ReaderBuilder::new()
7224            .with_projection(vec![1])
7225            .build(Cursor::new(mutated))
7226            .expect("build projected reader")
7227            .collect::<Result<Vec<_>, _>>()
7228            .expect_err("projection must also reject skipped int overflow");
7229        assert!(matches!(err, ArrowError::AvroError(_)));
7230        assert!(err.to_string().contains("varint overflow"));
7231    }
7232
7233    #[test]
7234    fn comprehensive_e2e_test() {
7235        let path = "test/data/comprehensive_e2e.avro";
7236        let batch = read_file(path, 1024, false);
7237        let schema = batch.schema();
7238
7239        #[inline]
7240        fn tid_by_name(fields: &UnionFields, want: &str) -> i8 {
7241            for (tid, f) in fields.iter() {
7242                if f.name() == want {
7243                    return tid;
7244                }
7245            }
7246            panic!("union child '{want}' not found");
7247        }
7248
7249        #[inline]
7250        fn tid_by_dt(fields: &UnionFields, pred: impl Fn(&DataType) -> bool) -> i8 {
7251            for (tid, f) in fields.iter() {
7252                if pred(f.data_type()) {
7253                    return tid;
7254                }
7255            }
7256            panic!("no union child matches predicate");
7257        }
7258
7259        fn mk_dense_union(
7260            fields: &UnionFields,
7261            type_ids: Vec<i8>,
7262            offsets: Vec<i32>,
7263            provide: impl Fn(&Field) -> Option<ArrayRef>,
7264        ) -> ArrayRef {
7265            fn empty_child_for(dt: &DataType) -> Arc<dyn Array> {
7266                match dt {
7267                    DataType::Null => Arc::new(NullArray::new(0)),
7268                    DataType::Boolean => Arc::new(BooleanArray::from(Vec::<bool>::new())),
7269                    DataType::Int32 => Arc::new(Int32Array::from(Vec::<i32>::new())),
7270                    DataType::Int64 => Arc::new(Int64Array::from(Vec::<i64>::new())),
7271                    DataType::Float32 => Arc::new(Float32Array::from(Vec::<f32>::new())),
7272                    DataType::Float64 => Arc::new(Float64Array::from(Vec::<f64>::new())),
7273                    DataType::Binary => Arc::new(BinaryArray::from(Vec::<&[u8]>::new())),
7274                    DataType::Utf8 => Arc::new(StringArray::from(Vec::<&str>::new())),
7275                    DataType::Date32 => Arc::new(Date32Array::from(Vec::<i32>::new())),
7276                    DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
7277                        Arc::new(Time32MillisecondArray::from(Vec::<i32>::new()))
7278                    }
7279                    DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
7280                        Arc::new(Time64MicrosecondArray::from(Vec::<i64>::new()))
7281                    }
7282                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
7283                        let a = TimestampMillisecondArray::from(Vec::<i64>::new());
7284                        Arc::new(if let Some(tz) = tz {
7285                            a.with_timezone(tz.clone())
7286                        } else {
7287                            a
7288                        })
7289                    }
7290                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
7291                        let a = TimestampMicrosecondArray::from(Vec::<i64>::new());
7292                        Arc::new(if let Some(tz) = tz {
7293                            a.with_timezone(tz.clone())
7294                        } else {
7295                            a
7296                        })
7297                    }
7298                    DataType::Interval(IntervalUnit::MonthDayNano) => Arc::new(
7299                        IntervalMonthDayNanoArray::from(Vec::<IntervalMonthDayNano>::new()),
7300                    ),
7301                    DataType::FixedSizeBinary(sz) => Arc::new(
7302                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(
7303                            std::iter::empty::<Option<Vec<u8>>>(),
7304                            *sz,
7305                        )
7306                        .unwrap(),
7307                    ),
7308                    DataType::Dictionary(_, _) => {
7309                        let keys = Int32Array::from(Vec::<i32>::new());
7310                        let values = Arc::new(StringArray::from(Vec::<&str>::new()));
7311                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
7312                    }
7313                    DataType::Struct(fields) => {
7314                        let children: Vec<ArrayRef> = fields
7315                            .iter()
7316                            .map(|f| empty_child_for(f.data_type()) as ArrayRef)
7317                            .collect();
7318                        Arc::new(StructArray::new(fields.clone(), children, None))
7319                    }
7320                    DataType::List(field) => {
7321                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
7322                        Arc::new(
7323                            ListArray::try_new(
7324                                field.clone(),
7325                                offsets,
7326                                empty_child_for(field.data_type()),
7327                                None,
7328                            )
7329                            .unwrap(),
7330                        )
7331                    }
7332                    DataType::Map(entry_field, is_sorted) => {
7333                        let (key_field, val_field) = match entry_field.data_type() {
7334                            DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
7335                            other => panic!("unexpected map entries type: {other:?}"),
7336                        };
7337                        let keys = StringArray::from(Vec::<&str>::new());
7338                        let vals: ArrayRef = match val_field.data_type() {
7339                            DataType::Null => Arc::new(NullArray::new(0)) as ArrayRef,
7340                            DataType::Boolean => {
7341                                Arc::new(BooleanArray::from(Vec::<bool>::new())) as ArrayRef
7342                            }
7343                            DataType::Int32 => {
7344                                Arc::new(Int32Array::from(Vec::<i32>::new())) as ArrayRef
7345                            }
7346                            DataType::Int64 => {
7347                                Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
7348                            }
7349                            DataType::Float32 => {
7350                                Arc::new(Float32Array::from(Vec::<f32>::new())) as ArrayRef
7351                            }
7352                            DataType::Float64 => {
7353                                Arc::new(Float64Array::from(Vec::<f64>::new())) as ArrayRef
7354                            }
7355                            DataType::Utf8 => {
7356                                Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
7357                            }
7358                            DataType::Binary => {
7359                                Arc::new(BinaryArray::from(Vec::<&[u8]>::new())) as ArrayRef
7360                            }
7361                            DataType::Union(uf, _) => {
7362                                let children: Vec<ArrayRef> = uf
7363                                    .iter()
7364                                    .map(|(_, f)| empty_child_for(f.data_type()))
7365                                    .collect();
7366                                Arc::new(
7367                                    UnionArray::try_new(
7368                                        uf.clone(),
7369                                        ScalarBuffer::<i8>::from(Vec::<i8>::new()),
7370                                        Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
7371                                        children,
7372                                    )
7373                                    .unwrap(),
7374                                ) as ArrayRef
7375                            }
7376                            other => panic!("unsupported map value type: {other:?}"),
7377                        };
7378                        let entries = StructArray::new(
7379                            Fields::from(vec![
7380                                key_field.as_ref().clone(),
7381                                val_field.as_ref().clone(),
7382                            ]),
7383                            vec![Arc::new(keys) as ArrayRef, vals],
7384                            None,
7385                        );
7386                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
7387                        Arc::new(MapArray::new(
7388                            entry_field.clone(),
7389                            offsets,
7390                            entries,
7391                            None,
7392                            *is_sorted,
7393                        ))
7394                    }
7395                    other => panic!("empty_child_for: unhandled type {other:?}"),
7396                }
7397            }
7398            let children: Vec<ArrayRef> = fields
7399                .iter()
7400                .map(|(_, f)| provide(f).unwrap_or_else(|| empty_child_for(f.data_type())))
7401                .collect();
7402            Arc::new(
7403                UnionArray::try_new(
7404                    fields.clone(),
7405                    ScalarBuffer::<i8>::from(type_ids),
7406                    Some(ScalarBuffer::<i32>::from(offsets)),
7407                    children,
7408                )
7409                .unwrap(),
7410            ) as ArrayRef
7411        }
7412
7413        #[inline]
7414        fn uuid16_from_str(s: &str) -> [u8; 16] {
7415            let mut out = [0u8; 16];
7416            let mut idx = 0usize;
7417            let mut hi: Option<u8> = None;
7418            for ch in s.chars() {
7419                if ch == '-' {
7420                    continue;
7421                }
7422                let v = ch.to_digit(16).expect("invalid hex digit in UUID") as u8;
7423                if let Some(h) = hi {
7424                    out[idx] = (h << 4) | v;
7425                    idx += 1;
7426                    hi = None;
7427                } else {
7428                    hi = Some(v);
7429                }
7430            }
7431            assert_eq!(idx, 16, "UUID must decode to 16 bytes");
7432            out
7433        }
7434        let date_a: i32 = 19_000; // 2022-01-08
7435        let time_ms_a: i32 = 12 * 3_600_000 + 34 * 60_000 + 56_000 + 789;
7436        let time_us_eod: i64 = 86_400_000_000 - 1;
7437        let ts_ms_2024_01_01: i64 = 1_704_067_200_000; // 2024-01-01T00:00:00Z
7438        let ts_us_2024_01_01: i64 = ts_ms_2024_01_01 * 1_000;
7439        let dur_small = IntervalMonthDayNanoType::make_value(1, 2, 3_000_000_000);
7440        let dur_zero = IntervalMonthDayNanoType::make_value(0, 0, 0);
7441        let dur_large =
7442            IntervalMonthDayNanoType::make_value(12, 31, ((86_400_000 - 1) as i64) * 1_000_000);
7443        let dur_2years = IntervalMonthDayNanoType::make_value(24, 0, 0);
7444        let uuid1 = uuid16_from_str("fe7bc30b-4ce8-4c5e-b67c-2234a2d38e66");
7445        let uuid2 = uuid16_from_str("0826cc06-d2e3-4599-b4ad-af5fa6905cdb");
7446
7447        #[inline]
7448        fn push_like(
7449            reader_schema: &arrow_schema::Schema,
7450            name: &str,
7451            arr: ArrayRef,
7452            fields: &mut Vec<FieldRef>,
7453            cols: &mut Vec<ArrayRef>,
7454        ) {
7455            let src = reader_schema
7456                .field_with_name(name)
7457                .unwrap_or_else(|_| panic!("source schema missing field '{name}'"));
7458            let mut f = Field::new(name, arr.data_type().clone(), src.is_nullable());
7459            let md = src.metadata();
7460            if !md.is_empty() {
7461                f = f.with_metadata(md.clone());
7462            }
7463            fields.push(Arc::new(f));
7464            cols.push(arr);
7465        }
7466
7467        let mut fields: Vec<FieldRef> = Vec::new();
7468        let mut columns: Vec<ArrayRef> = Vec::new();
7469        push_like(
7470            schema.as_ref(),
7471            "id",
7472            Arc::new(Int64Array::from(vec![1, 2, 3, 4])) as ArrayRef,
7473            &mut fields,
7474            &mut columns,
7475        );
7476        push_like(
7477            schema.as_ref(),
7478            "flag",
7479            Arc::new(BooleanArray::from(vec![true, false, true, false])) as ArrayRef,
7480            &mut fields,
7481            &mut columns,
7482        );
7483        push_like(
7484            schema.as_ref(),
7485            "ratio_f32",
7486            Arc::new(Float32Array::from(vec![1.25f32, -0.0, 3.5, 9.75])) as ArrayRef,
7487            &mut fields,
7488            &mut columns,
7489        );
7490        push_like(
7491            schema.as_ref(),
7492            "ratio_f64",
7493            Arc::new(Float64Array::from(vec![2.5f64, -1.0, 7.0, -2.25])) as ArrayRef,
7494            &mut fields,
7495            &mut columns,
7496        );
7497        push_like(
7498            schema.as_ref(),
7499            "count_i32",
7500            Arc::new(Int32Array::from(vec![7, -1, 0, 123])) as ArrayRef,
7501            &mut fields,
7502            &mut columns,
7503        );
7504        push_like(
7505            schema.as_ref(),
7506            "count_i64",
7507            Arc::new(Int64Array::from(vec![
7508                7_000_000_000i64,
7509                -2,
7510                0,
7511                -9_876_543_210i64,
7512            ])) as ArrayRef,
7513            &mut fields,
7514            &mut columns,
7515        );
7516        push_like(
7517            schema.as_ref(),
7518            "opt_i32_nullfirst",
7519            Arc::new(Int32Array::from(vec![None, Some(42), None, Some(0)])) as ArrayRef,
7520            &mut fields,
7521            &mut columns,
7522        );
7523        push_like(
7524            schema.as_ref(),
7525            "opt_str_nullsecond",
7526            Arc::new(StringArray::from(vec![
7527                Some("alpha"),
7528                None,
7529                Some("s3"),
7530                Some(""),
7531            ])) as ArrayRef,
7532            &mut fields,
7533            &mut columns,
7534        );
7535        {
7536            let uf = match schema
7537                .field_with_name("tri_union_prim")
7538                .unwrap()
7539                .data_type()
7540            {
7541                DataType::Union(f, UnionMode::Dense) => f.clone(),
7542                other => panic!("tri_union_prim should be dense union, got {other:?}"),
7543            };
7544            let tid_i = tid_by_name(&uf, "int");
7545            let tid_s = tid_by_name(&uf, "string");
7546            let tid_b = tid_by_name(&uf, "boolean");
7547            let tids = vec![tid_i, tid_s, tid_b, tid_s];
7548            let offs = vec![0, 0, 0, 1];
7549            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7550                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![0])) as ArrayRef),
7551                DataType::Utf8 => Some(Arc::new(StringArray::from(vec!["hi", ""])) as ArrayRef),
7552                DataType::Boolean => Some(Arc::new(BooleanArray::from(vec![true])) as ArrayRef),
7553                _ => None,
7554            });
7555            push_like(
7556                schema.as_ref(),
7557                "tri_union_prim",
7558                arr,
7559                &mut fields,
7560                &mut columns,
7561            );
7562        }
7563
7564        push_like(
7565            schema.as_ref(),
7566            "str_utf8",
7567            Arc::new(StringArray::from(vec!["hello", "", "world", "✓ unicode"])) as ArrayRef,
7568            &mut fields,
7569            &mut columns,
7570        );
7571        push_like(
7572            schema.as_ref(),
7573            "raw_bytes",
7574            Arc::new(BinaryArray::from(vec![
7575                b"\x00\x01".as_ref(),
7576                b"".as_ref(),
7577                b"\xFF\x00".as_ref(),
7578                b"\x10\x20\x30\x40".as_ref(),
7579            ])) as ArrayRef,
7580            &mut fields,
7581            &mut columns,
7582        );
7583        {
7584            let it = [
7585                Some(*b"0123456789ABCDEF"),
7586                Some([0u8; 16]),
7587                Some(*b"ABCDEFGHIJKLMNOP"),
7588                Some([0xAA; 16]),
7589            ]
7590            .into_iter();
7591            let arr =
7592                Arc::new(FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap())
7593                    as ArrayRef;
7594            push_like(
7595                schema.as_ref(),
7596                "fx16_plain",
7597                arr,
7598                &mut fields,
7599                &mut columns,
7600            );
7601        }
7602        {
7603            #[cfg(feature = "small_decimals")]
7604            let dec10_2 = Arc::new(
7605                Decimal64Array::from_iter_values([123456i64, -1, 0, 9_999_999_999i64])
7606                    .with_precision_and_scale(10, 2)
7607                    .unwrap(),
7608            ) as ArrayRef;
7609            #[cfg(not(feature = "small_decimals"))]
7610            let dec10_2 = Arc::new(
7611                Decimal128Array::from_iter_values([123456i128, -1, 0, 9_999_999_999i128])
7612                    .with_precision_and_scale(10, 2)
7613                    .unwrap(),
7614            ) as ArrayRef;
7615            push_like(
7616                schema.as_ref(),
7617                "dec_bytes_s10_2",
7618                dec10_2,
7619                &mut fields,
7620                &mut columns,
7621            );
7622        }
7623        {
7624            #[cfg(feature = "small_decimals")]
7625            let dec20_4 = Arc::new(
7626                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
7627                    .with_precision_and_scale(20, 4)
7628                    .unwrap(),
7629            ) as ArrayRef;
7630            #[cfg(not(feature = "small_decimals"))]
7631            let dec20_4 = Arc::new(
7632                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
7633                    .with_precision_and_scale(20, 4)
7634                    .unwrap(),
7635            ) as ArrayRef;
7636            push_like(
7637                schema.as_ref(),
7638                "dec_fix_s20_4",
7639                dec20_4,
7640                &mut fields,
7641                &mut columns,
7642            );
7643        }
7644        {
7645            let it = [Some(uuid1), Some(uuid2), Some(uuid1), Some(uuid2)].into_iter();
7646            let arr =
7647                Arc::new(FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap())
7648                    as ArrayRef;
7649            push_like(schema.as_ref(), "uuid_str", arr, &mut fields, &mut columns);
7650        }
7651        push_like(
7652            schema.as_ref(),
7653            "d_date",
7654            Arc::new(Date32Array::from(vec![date_a, 0, 1, 365])) as ArrayRef,
7655            &mut fields,
7656            &mut columns,
7657        );
7658        push_like(
7659            schema.as_ref(),
7660            "t_millis",
7661            Arc::new(Time32MillisecondArray::from(vec![
7662                time_ms_a,
7663                0,
7664                1,
7665                86_400_000 - 1,
7666            ])) as ArrayRef,
7667            &mut fields,
7668            &mut columns,
7669        );
7670        push_like(
7671            schema.as_ref(),
7672            "t_micros",
7673            Arc::new(Time64MicrosecondArray::from(vec![
7674                time_us_eod,
7675                0,
7676                1,
7677                1_000_000,
7678            ])) as ArrayRef,
7679            &mut fields,
7680            &mut columns,
7681        );
7682        {
7683            let a = TimestampMillisecondArray::from(vec![
7684                ts_ms_2024_01_01,
7685                -1,
7686                ts_ms_2024_01_01 + 123,
7687                0,
7688            ])
7689            .with_timezone("+00:00");
7690            push_like(
7691                schema.as_ref(),
7692                "ts_millis_utc",
7693                Arc::new(a) as ArrayRef,
7694                &mut fields,
7695                &mut columns,
7696            );
7697        }
7698        {
7699            let a = TimestampMicrosecondArray::from(vec![
7700                ts_us_2024_01_01,
7701                1,
7702                ts_us_2024_01_01 + 456,
7703                0,
7704            ])
7705            .with_timezone("+00:00");
7706            push_like(
7707                schema.as_ref(),
7708                "ts_micros_utc",
7709                Arc::new(a) as ArrayRef,
7710                &mut fields,
7711                &mut columns,
7712            );
7713        }
7714        push_like(
7715            schema.as_ref(),
7716            "ts_millis_local",
7717            Arc::new(TimestampMillisecondArray::from(vec![
7718                ts_ms_2024_01_01 + 86_400_000,
7719                0,
7720                ts_ms_2024_01_01 + 789,
7721                123_456_789,
7722            ])) as ArrayRef,
7723            &mut fields,
7724            &mut columns,
7725        );
7726        push_like(
7727            schema.as_ref(),
7728            "ts_micros_local",
7729            Arc::new(TimestampMicrosecondArray::from(vec![
7730                ts_us_2024_01_01 + 123_456,
7731                0,
7732                ts_us_2024_01_01 + 101_112,
7733                987_654_321,
7734            ])) as ArrayRef,
7735            &mut fields,
7736            &mut columns,
7737        );
7738        {
7739            let v = vec![dur_small, dur_zero, dur_large, dur_2years];
7740            push_like(
7741                schema.as_ref(),
7742                "interval_mdn",
7743                Arc::new(IntervalMonthDayNanoArray::from(v)) as ArrayRef,
7744                &mut fields,
7745                &mut columns,
7746            );
7747        }
7748        {
7749            let keys = Int32Array::from(vec![1, 2, 3, 0]); // NEW, PROCESSING, DONE, UNKNOWN
7750            let values = Arc::new(StringArray::from(vec![
7751                "UNKNOWN",
7752                "NEW",
7753                "PROCESSING",
7754                "DONE",
7755            ])) as ArrayRef;
7756            let dict = DictionaryArray::<Int32Type>::try_new(keys, values).unwrap();
7757            push_like(
7758                schema.as_ref(),
7759                "status",
7760                Arc::new(dict) as ArrayRef,
7761                &mut fields,
7762                &mut columns,
7763            );
7764        }
7765        {
7766            let list_field = match schema.field_with_name("arr_union").unwrap().data_type() {
7767                DataType::List(f) => f.clone(),
7768                other => panic!("arr_union should be List, got {other:?}"),
7769            };
7770            let uf = match list_field.data_type() {
7771                DataType::Union(f, UnionMode::Dense) => f.clone(),
7772                other => panic!("arr_union item should be union, got {other:?}"),
7773            };
7774            let tid_l = tid_by_name(&uf, "long");
7775            let tid_s = tid_by_name(&uf, "string");
7776            let tid_n = tid_by_name(&uf, "null");
7777            let type_ids = vec![
7778                tid_l, tid_s, tid_n, tid_l, tid_n, tid_s, tid_l, tid_l, tid_s, tid_n, tid_l,
7779            ];
7780            let offsets = vec![0, 0, 0, 1, 1, 1, 2, 3, 2, 2, 4];
7781            let values = mk_dense_union(&uf, type_ids, offsets, |f| match f.data_type() {
7782                DataType::Int64 => {
7783                    Some(Arc::new(Int64Array::from(vec![1i64, -3, 0, -1, 0])) as ArrayRef)
7784                }
7785                DataType::Utf8 => {
7786                    Some(Arc::new(StringArray::from(vec!["x", "z", "end"])) as ArrayRef)
7787                }
7788                DataType::Null => Some(Arc::new(NullArray::new(3)) as ArrayRef),
7789                _ => None,
7790            });
7791            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 4, 7, 8, 11]));
7792            let arr = Arc::new(ListArray::try_new(list_field, list_offsets, values, None).unwrap())
7793                as ArrayRef;
7794            push_like(schema.as_ref(), "arr_union", arr, &mut fields, &mut columns);
7795        }
7796        {
7797            let (entry_field, entries_fields, uf, is_sorted) =
7798                match schema.field_with_name("map_union").unwrap().data_type() {
7799                    DataType::Map(entry_field, is_sorted) => {
7800                        let fs = match entry_field.data_type() {
7801                            DataType::Struct(fs) => fs.clone(),
7802                            other => panic!("map entries must be struct, got {other:?}"),
7803                        };
7804                        let val_f = fs[1].clone();
7805                        let uf = match val_f.data_type() {
7806                            DataType::Union(f, UnionMode::Dense) => f.clone(),
7807                            other => panic!("map value must be union, got {other:?}"),
7808                        };
7809                        (entry_field.clone(), fs, uf, *is_sorted)
7810                    }
7811                    other => panic!("map_union should be Map, got {other:?}"),
7812                };
7813            let keys = StringArray::from(vec!["a", "b", "c", "neg", "pi", "ok"]);
7814            let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4, 4, 6]));
7815            let tid_null = tid_by_name(&uf, "null");
7816            let tid_d = tid_by_name(&uf, "double");
7817            let tid_s = tid_by_name(&uf, "string");
7818            let type_ids = vec![tid_d, tid_null, tid_s, tid_d, tid_d, tid_s];
7819            let offsets = vec![0, 0, 0, 1, 2, 1];
7820            let pi_5dp = (std::f64::consts::PI * 100_000.0).trunc() / 100_000.0;
7821            let vals = mk_dense_union(&uf, type_ids, offsets, |f| match f.data_type() {
7822                DataType::Float64 => {
7823                    Some(Arc::new(Float64Array::from(vec![1.5f64, -0.5, pi_5dp])) as ArrayRef)
7824                }
7825                DataType::Utf8 => {
7826                    Some(Arc::new(StringArray::from(vec!["yes", "true"])) as ArrayRef)
7827                }
7828                DataType::Null => Some(Arc::new(NullArray::new(2)) as ArrayRef),
7829                _ => None,
7830            });
7831            let entries = StructArray::new(
7832                entries_fields.clone(),
7833                vec![Arc::new(keys) as ArrayRef, vals],
7834                None,
7835            );
7836            let map =
7837                Arc::new(MapArray::new(entry_field, moff, entries, None, is_sorted)) as ArrayRef;
7838            push_like(schema.as_ref(), "map_union", map, &mut fields, &mut columns);
7839        }
7840        {
7841            let fs = match schema.field_with_name("address").unwrap().data_type() {
7842                DataType::Struct(fs) => fs.clone(),
7843                other => panic!("address should be Struct, got {other:?}"),
7844            };
7845            let street = Arc::new(StringArray::from(vec![
7846                "100 Main",
7847                "",
7848                "42 Galaxy Way",
7849                "End Ave",
7850            ])) as ArrayRef;
7851            let zip = Arc::new(Int32Array::from(vec![12345, 0, 42424, 1])) as ArrayRef;
7852            let country = Arc::new(StringArray::from(vec!["US", "CA", "US", "GB"])) as ArrayRef;
7853            let arr = Arc::new(StructArray::new(fs, vec![street, zip, country], None)) as ArrayRef;
7854            push_like(schema.as_ref(), "address", arr, &mut fields, &mut columns);
7855        }
7856        {
7857            let fs = match schema.field_with_name("maybe_auth").unwrap().data_type() {
7858                DataType::Struct(fs) => fs.clone(),
7859                other => panic!("maybe_auth should be Struct, got {other:?}"),
7860            };
7861            let user =
7862                Arc::new(StringArray::from(vec!["alice", "bob", "carol", "dave"])) as ArrayRef;
7863            let token_values: Vec<Option<&[u8]>> = vec![
7864                None,                           // row 1: null
7865                Some(b"\x01\x02\x03".as_ref()), // row 2: bytes
7866                None,                           // row 3: null
7867                Some(b"".as_ref()),             // row 4: empty bytes
7868            ];
7869            let token = Arc::new(BinaryArray::from(token_values)) as ArrayRef;
7870            let arr = Arc::new(StructArray::new(fs, vec![user, token], None)) as ArrayRef;
7871            push_like(
7872                schema.as_ref(),
7873                "maybe_auth",
7874                arr,
7875                &mut fields,
7876                &mut columns,
7877            );
7878        }
7879        {
7880            let uf = match schema
7881                .field_with_name("union_enum_record_array_map")
7882                .unwrap()
7883                .data_type()
7884            {
7885                DataType::Union(f, UnionMode::Dense) => f.clone(),
7886                other => panic!("union_enum_record_array_map should be union, got {other:?}"),
7887            };
7888            let mut tid_enum: Option<i8> = None;
7889            let mut tid_rec_a: Option<i8> = None;
7890            let mut tid_array: Option<i8> = None;
7891            let mut tid_map: Option<i8> = None;
7892            let mut map_entry_field: Option<FieldRef> = None;
7893            let mut map_sorted: bool = false;
7894            for (tid, f) in uf.iter() {
7895                match f.data_type() {
7896                    DataType::Dictionary(_, _) => tid_enum = Some(tid),
7897                    DataType::Struct(childs)
7898                        if childs.len() == 2
7899                            && childs[0].name() == "a"
7900                            && childs[1].name() == "b" =>
7901                    {
7902                        tid_rec_a = Some(tid)
7903                    }
7904                    DataType::List(item) if matches!(item.data_type(), DataType::Int64) => {
7905                        tid_array = Some(tid)
7906                    }
7907                    DataType::Map(ef, is_sorted) => {
7908                        tid_map = Some(tid);
7909                        map_entry_field = Some(ef.clone());
7910                        map_sorted = *is_sorted;
7911                    }
7912                    _ => {}
7913                }
7914            }
7915            let (tid_enum, tid_rec_a, tid_array, tid_map) = (
7916                tid_enum.unwrap(),
7917                tid_rec_a.unwrap(),
7918                tid_array.unwrap(),
7919                tid_map.unwrap(),
7920            );
7921            let tids = vec![tid_enum, tid_rec_a, tid_array, tid_map];
7922            let offs = vec![0, 0, 0, 0];
7923            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7924                DataType::Dictionary(_, _) => {
7925                    let keys = Int32Array::from(vec![0i32]);
7926                    let values =
7927                        Arc::new(StringArray::from(vec!["RED", "GREEN", "BLUE"])) as ArrayRef;
7928                    Some(
7929                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
7930                            as ArrayRef,
7931                    )
7932                }
7933                DataType::Struct(fs)
7934                    if fs.len() == 2 && fs[0].name() == "a" && fs[1].name() == "b" =>
7935                {
7936                    let a = Int32Array::from(vec![7]);
7937                    let b = StringArray::from(vec!["rec"]);
7938                    Some(Arc::new(StructArray::new(
7939                        fs.clone(),
7940                        vec![Arc::new(a), Arc::new(b)],
7941                        None,
7942                    )) as ArrayRef)
7943                }
7944                DataType::List(field) => {
7945                    let values = Int64Array::from(vec![1i64, 2, 3]);
7946                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3]));
7947                    Some(Arc::new(
7948                        ListArray::try_new(field.clone(), offsets, Arc::new(values), None).unwrap(),
7949                    ) as ArrayRef)
7950                }
7951                DataType::Map(_, _) => {
7952                    let entry_field = map_entry_field.clone().unwrap();
7953                    let (key_field, val_field) = match entry_field.data_type() {
7954                        DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
7955                        _ => unreachable!(),
7956                    };
7957                    let keys = StringArray::from(vec!["k"]);
7958                    let vals = StringArray::from(vec!["v"]);
7959                    let entries = StructArray::new(
7960                        Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
7961                        vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
7962                        None,
7963                    );
7964                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 1]));
7965                    Some(Arc::new(MapArray::new(
7966                        entry_field.clone(),
7967                        offsets,
7968                        entries,
7969                        None,
7970                        map_sorted,
7971                    )) as ArrayRef)
7972                }
7973                _ => None,
7974            });
7975            push_like(
7976                schema.as_ref(),
7977                "union_enum_record_array_map",
7978                arr,
7979                &mut fields,
7980                &mut columns,
7981            );
7982        }
7983        {
7984            let uf = match schema
7985                .field_with_name("union_date_or_fixed4")
7986                .unwrap()
7987                .data_type()
7988            {
7989                DataType::Union(f, UnionMode::Dense) => f.clone(),
7990                other => panic!("union_date_or_fixed4 should be union, got {other:?}"),
7991            };
7992            let tid_date = tid_by_dt(&uf, |dt| matches!(dt, DataType::Date32));
7993            let tid_fx4 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(4)));
7994            let tids = vec![tid_date, tid_fx4, tid_date, tid_fx4];
7995            let offs = vec![0, 0, 1, 1];
7996            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
7997                DataType::Date32 => Some(Arc::new(Date32Array::from(vec![date_a, 0])) as ArrayRef),
7998                DataType::FixedSizeBinary(4) => {
7999                    let it = [Some(*b"\x00\x11\x22\x33"), Some(*b"ABCD")].into_iter();
8000                    Some(Arc::new(
8001                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 4).unwrap(),
8002                    ) as ArrayRef)
8003                }
8004                _ => None,
8005            });
8006            push_like(
8007                schema.as_ref(),
8008                "union_date_or_fixed4",
8009                arr,
8010                &mut fields,
8011                &mut columns,
8012            );
8013        }
8014        {
8015            let uf = match schema
8016                .field_with_name("union_interval_or_string")
8017                .unwrap()
8018                .data_type()
8019            {
8020                DataType::Union(f, UnionMode::Dense) => f.clone(),
8021                other => panic!("union_interval_or_string should be union, got {other:?}"),
8022            };
8023            let tid_dur = tid_by_dt(&uf, |dt| {
8024                matches!(dt, DataType::Interval(IntervalUnit::MonthDayNano))
8025            });
8026            let tid_str = tid_by_dt(&uf, |dt| matches!(dt, DataType::Utf8));
8027            let tids = vec![tid_dur, tid_str, tid_dur, tid_str];
8028            let offs = vec![0, 0, 1, 1];
8029            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
8030                DataType::Interval(IntervalUnit::MonthDayNano) => Some(Arc::new(
8031                    IntervalMonthDayNanoArray::from(vec![dur_small, dur_large]),
8032                )
8033                    as ArrayRef),
8034                DataType::Utf8 => Some(Arc::new(StringArray::from(vec![
8035                    "duration-as-text",
8036                    "iso-8601-period-P1Y",
8037                ])) as ArrayRef),
8038                _ => None,
8039            });
8040            push_like(
8041                schema.as_ref(),
8042                "union_interval_or_string",
8043                arr,
8044                &mut fields,
8045                &mut columns,
8046            );
8047        }
8048        {
8049            let uf = match schema
8050                .field_with_name("union_uuid_or_fixed10")
8051                .unwrap()
8052                .data_type()
8053            {
8054                DataType::Union(f, UnionMode::Dense) => f.clone(),
8055                other => panic!("union_uuid_or_fixed10 should be union, got {other:?}"),
8056            };
8057            let tid_uuid = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(16)));
8058            let tid_fx10 = tid_by_dt(&uf, |dt| matches!(dt, DataType::FixedSizeBinary(10)));
8059            let tids = vec![tid_uuid, tid_fx10, tid_uuid, tid_fx10];
8060            let offs = vec![0, 0, 1, 1];
8061            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
8062                DataType::FixedSizeBinary(16) => {
8063                    let it = [Some(uuid1), Some(uuid2)].into_iter();
8064                    Some(Arc::new(
8065                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
8066                    ) as ArrayRef)
8067                }
8068                DataType::FixedSizeBinary(10) => {
8069                    let fx10_a = [0xAAu8; 10];
8070                    let fx10_b = [0x00u8, 0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88, 0x99];
8071                    let it = [Some(fx10_a), Some(fx10_b)].into_iter();
8072                    Some(Arc::new(
8073                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 10).unwrap(),
8074                    ) as ArrayRef)
8075                }
8076                _ => None,
8077            });
8078            push_like(
8079                schema.as_ref(),
8080                "union_uuid_or_fixed10",
8081                arr,
8082                &mut fields,
8083                &mut columns,
8084            );
8085        }
8086        {
8087            let list_field = match schema
8088                .field_with_name("array_records_with_union")
8089                .unwrap()
8090                .data_type()
8091            {
8092                DataType::List(f) => f.clone(),
8093                other => panic!("array_records_with_union should be List, got {other:?}"),
8094            };
8095            let kv_fields = match list_field.data_type() {
8096                DataType::Struct(fs) => fs.clone(),
8097                other => panic!("array_records_with_union items must be Struct, got {other:?}"),
8098            };
8099            let val_field = kv_fields
8100                .iter()
8101                .find(|f| f.name() == "val")
8102                .unwrap()
8103                .clone();
8104            let uf = match val_field.data_type() {
8105                DataType::Union(f, UnionMode::Dense) => f.clone(),
8106                other => panic!("KV.val should be union, got {other:?}"),
8107            };
8108            let keys = Arc::new(StringArray::from(vec!["k1", "k2", "k", "k3", "x"])) as ArrayRef;
8109            let tid_null = tid_by_name(&uf, "null");
8110            let tid_i = tid_by_name(&uf, "int");
8111            let tid_l = tid_by_name(&uf, "long");
8112            let type_ids = vec![tid_i, tid_null, tid_l, tid_null, tid_i];
8113            let offsets = vec![0, 0, 0, 1, 1];
8114            let vals = mk_dense_union(&uf, type_ids, offsets, |f| match f.data_type() {
8115                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![5, -5])) as ArrayRef),
8116                DataType::Int64 => Some(Arc::new(Int64Array::from(vec![99i64])) as ArrayRef),
8117                DataType::Null => Some(Arc::new(NullArray::new(2)) as ArrayRef),
8118                _ => None,
8119            });
8120            let values_struct =
8121                Arc::new(StructArray::new(kv_fields.clone(), vec![keys, vals], None)) as ArrayRef;
8122            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3, 4, 5]));
8123            let arr = Arc::new(
8124                ListArray::try_new(list_field, list_offsets, values_struct, None).unwrap(),
8125            ) as ArrayRef;
8126            push_like(
8127                schema.as_ref(),
8128                "array_records_with_union",
8129                arr,
8130                &mut fields,
8131                &mut columns,
8132            );
8133        }
8134        {
8135            let uf = match schema
8136                .field_with_name("union_map_or_array_int")
8137                .unwrap()
8138                .data_type()
8139            {
8140                DataType::Union(f, UnionMode::Dense) => f.clone(),
8141                other => panic!("union_map_or_array_int should be union, got {other:?}"),
8142            };
8143            let tid_map = tid_by_dt(&uf, |dt| matches!(dt, DataType::Map(_, _)));
8144            let tid_list = tid_by_dt(&uf, |dt| matches!(dt, DataType::List(_)));
8145            let map_child: ArrayRef = {
8146                let (entry_field, is_sorted) = match uf
8147                    .iter()
8148                    .find(|(tid, _)| *tid == tid_map)
8149                    .unwrap()
8150                    .1
8151                    .data_type()
8152                {
8153                    DataType::Map(ef, is_sorted) => (ef.clone(), *is_sorted),
8154                    _ => unreachable!(),
8155                };
8156                let (key_field, val_field) = match entry_field.data_type() {
8157                    DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
8158                    _ => unreachable!(),
8159                };
8160                let keys = StringArray::from(vec!["x", "y", "only"]);
8161                let vals = Int32Array::from(vec![1, 2, 10]);
8162                let entries = StructArray::new(
8163                    Fields::from(vec![key_field.as_ref().clone(), val_field.as_ref().clone()]),
8164                    vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
8165                    None,
8166                );
8167                let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3]));
8168                Arc::new(MapArray::new(entry_field, moff, entries, None, is_sorted)) as ArrayRef
8169            };
8170            let list_child: ArrayRef = {
8171                let list_field = match uf
8172                    .iter()
8173                    .find(|(tid, _)| *tid == tid_list)
8174                    .unwrap()
8175                    .1
8176                    .data_type()
8177                {
8178                    DataType::List(f) => f.clone(),
8179                    _ => unreachable!(),
8180                };
8181                let values = Int32Array::from(vec![1, 2, 3, 0]);
8182                let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4]));
8183                Arc::new(ListArray::try_new(list_field, offsets, Arc::new(values), None).unwrap())
8184                    as ArrayRef
8185            };
8186            let tids = vec![tid_map, tid_list, tid_map, tid_list];
8187            let offs = vec![0, 0, 1, 1];
8188            let arr = mk_dense_union(&uf, tids, offs, |f| match f.data_type() {
8189                DataType::Map(_, _) => Some(map_child.clone()),
8190                DataType::List(_) => Some(list_child.clone()),
8191                _ => None,
8192            });
8193            push_like(
8194                schema.as_ref(),
8195                "union_map_or_array_int",
8196                arr,
8197                &mut fields,
8198                &mut columns,
8199            );
8200        }
8201        push_like(
8202            schema.as_ref(),
8203            "renamed_with_default",
8204            Arc::new(Int32Array::from(vec![100, 42, 7, 42])) as ArrayRef,
8205            &mut fields,
8206            &mut columns,
8207        );
8208        {
8209            let fs = match schema.field_with_name("person").unwrap().data_type() {
8210                DataType::Struct(fs) => fs.clone(),
8211                other => panic!("person should be Struct, got {other:?}"),
8212            };
8213            let name =
8214                Arc::new(StringArray::from(vec!["Alice", "Bob", "Carol", "Dave"])) as ArrayRef;
8215            let age = Arc::new(Int32Array::from(vec![30, 0, 25, 41])) as ArrayRef;
8216            let arr = Arc::new(StructArray::new(fs, vec![name, age], None)) as ArrayRef;
8217            push_like(schema.as_ref(), "person", arr, &mut fields, &mut columns);
8218        }
8219        let expected =
8220            RecordBatch::try_new(Arc::new(Schema::new(Fields::from(fields))), columns).unwrap();
8221        assert_eq!(
8222            expected, batch,
8223            "entire RecordBatch mismatch (schema, all columns, all rows)"
8224        );
8225    }
8226    #[test]
8227    fn comprehensive_e2e_resolution_test() {
8228        use serde_json::Value;
8229        use std::collections::HashMap;
8230
8231        // Build a reader schema that stresses Avro schema‑resolution
8232        //
8233        // Changes relative to writer schema:
8234        // * Rename fields using writer aliases:    id -> identifier, renamed_with_default -> old_count
8235        // * Promote numeric types:                 count_i32 (int) -> long, ratio_f32 (float) -> double
8236        // * Reorder many union branches (reverse), incl. nested unions
8237        // * Reorder array/map union item/value branches
8238        // * Rename nested Address field:           street -> street_name (uses alias in writer)
8239        // * Change Person type name/namespace:     com.example.Person (matches writer alias)
8240        // * Reverse top‑level field order
8241        //
8242        // Reader‑side aliases are added wherever names change (per Avro spec).
8243        fn make_comprehensive_reader_schema(path: &str) -> AvroSchema {
8244            fn set_type_string(f: &mut Value, new_ty: &str) {
8245                if let Some(ty) = f.get_mut("type") {
8246                    match ty {
8247                        Value::String(_) | Value::Object(_) => {
8248                            *ty = Value::String(new_ty.to_string());
8249                        }
8250                        Value::Array(arr) => {
8251                            for b in arr.iter_mut() {
8252                                match b {
8253                                    Value::String(s) if s != "null" => {
8254                                        *b = Value::String(new_ty.to_string());
8255                                        break;
8256                                    }
8257                                    Value::Object(_) => {
8258                                        *b = Value::String(new_ty.to_string());
8259                                        break;
8260                                    }
8261                                    _ => {}
8262                                }
8263                            }
8264                        }
8265                        _ => {}
8266                    }
8267                }
8268            }
8269            fn reverse_union_array(f: &mut Value) {
8270                if let Some(arr) = f.get_mut("type").and_then(|t| t.as_array_mut()) {
8271                    arr.reverse();
8272                }
8273            }
8274            fn reverse_items_union(f: &mut Value) {
8275                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
8276                    if let Some(items) = obj.get_mut("items").and_then(|v| v.as_array_mut()) {
8277                        items.reverse();
8278                    }
8279                }
8280            }
8281            fn reverse_map_values_union(f: &mut Value) {
8282                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
8283                    if let Some(values) = obj.get_mut("values").and_then(|v| v.as_array_mut()) {
8284                        values.reverse();
8285                    }
8286                }
8287            }
8288            fn reverse_nested_union_in_record(f: &mut Value, field_name: &str) {
8289                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
8290                    if let Some(fields) = obj.get_mut("fields").and_then(|v| v.as_array_mut()) {
8291                        for ff in fields.iter_mut() {
8292                            if ff.get("name").and_then(|n| n.as_str()) == Some(field_name) {
8293                                if let Some(ty) = ff.get_mut("type") {
8294                                    if let Some(arr) = ty.as_array_mut() {
8295                                        arr.reverse();
8296                                    }
8297                                }
8298                            }
8299                        }
8300                    }
8301                }
8302            }
8303            fn rename_nested_field_with_alias(f: &mut Value, old: &str, new: &str) {
8304                if let Some(obj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
8305                    if let Some(fields) = obj.get_mut("fields").and_then(|v| v.as_array_mut()) {
8306                        for ff in fields.iter_mut() {
8307                            if ff.get("name").and_then(|n| n.as_str()) == Some(old) {
8308                                ff["name"] = Value::String(new.to_string());
8309                                ff["aliases"] = Value::Array(vec![Value::String(old.to_string())]);
8310                            }
8311                        }
8312                    }
8313                }
8314            }
8315            let mut root = load_writer_schema_json(path);
8316            assert_eq!(root["type"], "record", "writer schema must be a record");
8317            let fields = root
8318                .get_mut("fields")
8319                .and_then(|f| f.as_array_mut())
8320                .expect("record has fields");
8321            for f in fields.iter_mut() {
8322                let Some(name) = f.get("name").and_then(|n| n.as_str()) else {
8323                    continue;
8324                };
8325                match name {
8326                    // Field aliasing (reader‑side aliases added)
8327                    "id" => {
8328                        f["name"] = Value::String("identifier".into());
8329                        f["aliases"] = Value::Array(vec![Value::String("id".into())]);
8330                    }
8331                    "renamed_with_default" => {
8332                        f["name"] = Value::String("old_count".into());
8333                        f["aliases"] =
8334                            Value::Array(vec![Value::String("renamed_with_default".into())]);
8335                    }
8336                    // Promotions
8337                    "count_i32" => set_type_string(f, "long"),
8338                    "ratio_f32" => set_type_string(f, "double"),
8339                    // Union reorder (exercise resolution)
8340                    "opt_str_nullsecond" => reverse_union_array(f),
8341                    "union_enum_record_array_map" => reverse_union_array(f),
8342                    "union_date_or_fixed4" => reverse_union_array(f),
8343                    "union_interval_or_string" => reverse_union_array(f),
8344                    "union_uuid_or_fixed10" => reverse_union_array(f),
8345                    "union_map_or_array_int" => reverse_union_array(f),
8346                    "maybe_auth" => reverse_nested_union_in_record(f, "token"),
8347                    // Array/Map unions
8348                    "arr_union" => reverse_items_union(f),
8349                    "map_union" => reverse_map_values_union(f),
8350                    // Nested rename using reader‑side alias
8351                    "address" => rename_nested_field_with_alias(f, "street", "street_name"),
8352                    // Type‑name alias for nested record
8353                    "person" => {
8354                        if let Some(tobj) = f.get_mut("type").and_then(|t| t.as_object_mut()) {
8355                            tobj.insert("name".to_string(), Value::String("Person".into()));
8356                            tobj.insert(
8357                                "namespace".to_string(),
8358                                Value::String("com.example".into()),
8359                            );
8360                            tobj.insert(
8361                                "aliases".into(),
8362                                Value::Array(vec![
8363                                    Value::String("PersonV2".into()),
8364                                    Value::String("com.example.v2.PersonV2".into()),
8365                                ]),
8366                            );
8367                        }
8368                    }
8369                    _ => {}
8370                }
8371            }
8372            fields.reverse();
8373            AvroSchema::new(root.to_string())
8374        }
8375
8376        let path = "test/data/comprehensive_e2e.avro";
8377        let reader_schema = make_comprehensive_reader_schema(path);
8378        let batch = read_alltypes_with_reader_schema(path, reader_schema.clone());
8379
8380        const UUID_EXT_KEY: &str = "ARROW:extension:name";
8381        const UUID_LOGICAL_KEY: &str = "logicalType";
8382
8383        let uuid_md_top: Option<HashMap<String, String>> = batch
8384            .schema()
8385            .field_with_name("uuid_str")
8386            .ok()
8387            .and_then(|f| {
8388                let md = f.metadata();
8389                let has_ext = md.get(UUID_EXT_KEY).is_some();
8390                let is_uuid_logical = md
8391                    .get(UUID_LOGICAL_KEY)
8392                    .map(|v| v.trim_matches('"') == "uuid")
8393                    .unwrap_or(false);
8394                if has_ext || is_uuid_logical {
8395                    Some(md.clone())
8396                } else {
8397                    None
8398                }
8399            });
8400
8401        let uuid_md_union: Option<HashMap<String, String>> = batch
8402            .schema()
8403            .field_with_name("union_uuid_or_fixed10")
8404            .ok()
8405            .and_then(|f| match f.data_type() {
8406                DataType::Union(uf, _) => uf
8407                    .iter()
8408                    .find(|(_, child)| child.name() == "uuid")
8409                    .and_then(|(_, child)| {
8410                        let md = child.metadata();
8411                        let has_ext = md.get(UUID_EXT_KEY).is_some();
8412                        let is_uuid_logical = md
8413                            .get(UUID_LOGICAL_KEY)
8414                            .map(|v| v.trim_matches('"') == "uuid")
8415                            .unwrap_or(false);
8416                        if has_ext || is_uuid_logical {
8417                            Some(md.clone())
8418                        } else {
8419                            None
8420                        }
8421                    }),
8422                _ => None,
8423            });
8424
8425        let add_uuid_ext_top = |f: Field| -> Field {
8426            if let Some(md) = &uuid_md_top {
8427                f.with_metadata(md.clone())
8428            } else {
8429                f
8430            }
8431        };
8432        let add_uuid_ext_union = |f: Field| -> Field {
8433            if let Some(md) = &uuid_md_union {
8434                f.with_metadata(md.clone())
8435            } else {
8436                f
8437            }
8438        };
8439
8440        #[inline]
8441        fn uuid16_from_str(s: &str) -> [u8; 16] {
8442            let mut out = [0u8; 16];
8443            let mut idx = 0usize;
8444            let mut hi: Option<u8> = None;
8445            for ch in s.chars() {
8446                if ch == '-' {
8447                    continue;
8448                }
8449                let v = ch.to_digit(16).expect("invalid hex digit in UUID") as u8;
8450                if let Some(h) = hi {
8451                    out[idx] = (h << 4) | v;
8452                    idx += 1;
8453                    hi = None;
8454                } else {
8455                    hi = Some(v);
8456                }
8457            }
8458            assert_eq!(idx, 16, "UUID must decode to 16 bytes");
8459            out
8460        }
8461
8462        fn mk_dense_union(
8463            fields: &UnionFields,
8464            type_ids: Vec<i8>,
8465            offsets: Vec<i32>,
8466            provide: impl Fn(&Field) -> Option<ArrayRef>,
8467        ) -> ArrayRef {
8468            fn empty_child_for(dt: &DataType) -> Arc<dyn Array> {
8469                match dt {
8470                    DataType::Null => Arc::new(NullArray::new(0)),
8471                    DataType::Boolean => Arc::new(BooleanArray::from(Vec::<bool>::new())),
8472                    DataType::Int32 => Arc::new(Int32Array::from(Vec::<i32>::new())),
8473                    DataType::Int64 => Arc::new(Int64Array::from(Vec::<i64>::new())),
8474                    DataType::Float32 => Arc::new(Float32Array::from(Vec::<f32>::new())),
8475                    DataType::Float64 => Arc::new(Float64Array::from(Vec::<f64>::new())),
8476                    DataType::Binary => Arc::new(BinaryArray::from(Vec::<&[u8]>::new())),
8477                    DataType::Utf8 => Arc::new(StringArray::from(Vec::<&str>::new())),
8478                    DataType::Date32 => Arc::new(Date32Array::from(Vec::<i32>::new())),
8479                    DataType::Time32(arrow_schema::TimeUnit::Millisecond) => {
8480                        Arc::new(Time32MillisecondArray::from(Vec::<i32>::new()))
8481                    }
8482                    DataType::Time64(arrow_schema::TimeUnit::Microsecond) => {
8483                        Arc::new(Time64MicrosecondArray::from(Vec::<i64>::new()))
8484                    }
8485                    DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, tz) => {
8486                        let a = TimestampMillisecondArray::from(Vec::<i64>::new());
8487                        Arc::new(if let Some(tz) = tz {
8488                            a.with_timezone(tz.clone())
8489                        } else {
8490                            a
8491                        })
8492                    }
8493                    DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, tz) => {
8494                        let a = TimestampMicrosecondArray::from(Vec::<i64>::new());
8495                        Arc::new(if let Some(tz) = tz {
8496                            a.with_timezone(tz.clone())
8497                        } else {
8498                            a
8499                        })
8500                    }
8501                    DataType::Interval(IntervalUnit::MonthDayNano) => Arc::new(
8502                        IntervalMonthDayNanoArray::from(Vec::<IntervalMonthDayNano>::new()),
8503                    ),
8504                    DataType::FixedSizeBinary(sz) => Arc::new(
8505                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(
8506                            std::iter::empty::<Option<Vec<u8>>>(),
8507                            *sz,
8508                        )
8509                        .unwrap(),
8510                    ),
8511                    DataType::Dictionary(_, _) => {
8512                        let keys = Int32Array::from(Vec::<i32>::new());
8513                        let values = Arc::new(StringArray::from(Vec::<&str>::new()));
8514                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
8515                    }
8516                    DataType::Struct(fields) => {
8517                        let children: Vec<ArrayRef> = fields
8518                            .iter()
8519                            .map(|f| empty_child_for(f.data_type()) as ArrayRef)
8520                            .collect();
8521                        Arc::new(StructArray::new(fields.clone(), children, None))
8522                    }
8523                    DataType::List(field) => {
8524                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
8525                        Arc::new(
8526                            ListArray::try_new(
8527                                field.clone(),
8528                                offsets,
8529                                empty_child_for(field.data_type()),
8530                                None,
8531                            )
8532                            .unwrap(),
8533                        )
8534                    }
8535                    DataType::Map(entry_field, is_sorted) => {
8536                        let (key_field, val_field) = match entry_field.data_type() {
8537                            DataType::Struct(fs) => (fs[0].clone(), fs[1].clone()),
8538                            other => panic!("unexpected map entries type: {other:?}"),
8539                        };
8540                        let keys = StringArray::from(Vec::<&str>::new());
8541                        let vals: ArrayRef = match val_field.data_type() {
8542                            DataType::Null => Arc::new(NullArray::new(0)) as ArrayRef,
8543                            DataType::Boolean => {
8544                                Arc::new(BooleanArray::from(Vec::<bool>::new())) as ArrayRef
8545                            }
8546                            DataType::Int32 => {
8547                                Arc::new(Int32Array::from(Vec::<i32>::new())) as ArrayRef
8548                            }
8549                            DataType::Int64 => {
8550                                Arc::new(Int64Array::from(Vec::<i64>::new())) as ArrayRef
8551                            }
8552                            DataType::Float32 => {
8553                                Arc::new(Float32Array::from(Vec::<f32>::new())) as ArrayRef
8554                            }
8555                            DataType::Float64 => {
8556                                Arc::new(Float64Array::from(Vec::<f64>::new())) as ArrayRef
8557                            }
8558                            DataType::Utf8 => {
8559                                Arc::new(StringArray::from(Vec::<&str>::new())) as ArrayRef
8560                            }
8561                            DataType::Binary => {
8562                                Arc::new(BinaryArray::from(Vec::<&[u8]>::new())) as ArrayRef
8563                            }
8564                            DataType::Union(uf, _) => {
8565                                let children: Vec<ArrayRef> = uf
8566                                    .iter()
8567                                    .map(|(_, f)| empty_child_for(f.data_type()))
8568                                    .collect();
8569                                Arc::new(
8570                                    UnionArray::try_new(
8571                                        uf.clone(),
8572                                        ScalarBuffer::<i8>::from(Vec::<i8>::new()),
8573                                        Some(ScalarBuffer::<i32>::from(Vec::<i32>::new())),
8574                                        children,
8575                                    )
8576                                    .unwrap(),
8577                                ) as ArrayRef
8578                            }
8579                            other => panic!("unsupported map value type: {other:?}"),
8580                        };
8581                        let entries = StructArray::new(
8582                            Fields::from(vec![
8583                                key_field.as_ref().clone(),
8584                                val_field.as_ref().clone(),
8585                            ]),
8586                            vec![Arc::new(keys) as ArrayRef, vals],
8587                            None,
8588                        );
8589                        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0]));
8590                        Arc::new(MapArray::new(
8591                            entry_field.clone(),
8592                            offsets,
8593                            entries,
8594                            None,
8595                            *is_sorted,
8596                        ))
8597                    }
8598                    other => panic!("empty_child_for: unhandled type {other:?}"),
8599                }
8600            }
8601            let children: Vec<ArrayRef> = fields
8602                .iter()
8603                .map(|(_, f)| provide(f).unwrap_or_else(|| empty_child_for(f.data_type())))
8604                .collect();
8605            Arc::new(
8606                UnionArray::try_new(
8607                    fields.clone(),
8608                    ScalarBuffer::<i8>::from(type_ids),
8609                    Some(ScalarBuffer::<i32>::from(offsets)),
8610                    children,
8611                )
8612                .unwrap(),
8613            ) as ArrayRef
8614        }
8615        let date_a: i32 = 19_000; // 2022-01-08
8616        let time_ms_a: i32 = 12 * 3_600_000 + 34 * 60_000 + 56_000 + 789;
8617        let time_us_eod: i64 = 86_400_000_000 - 1;
8618        let ts_ms_2024_01_01: i64 = 1_704_067_200_000; // 2024-01-01T00:00:00Z
8619        let ts_us_2024_01_01: i64 = ts_ms_2024_01_01 * 1_000;
8620        let dur_small = IntervalMonthDayNanoType::make_value(1, 2, 3_000_000_000);
8621        let dur_zero = IntervalMonthDayNanoType::make_value(0, 0, 0);
8622        let dur_large =
8623            IntervalMonthDayNanoType::make_value(12, 31, ((86_400_000 - 1) as i64) * 1_000_000);
8624        let dur_2years = IntervalMonthDayNanoType::make_value(24, 0, 0);
8625        let uuid1 = uuid16_from_str("fe7bc30b-4ce8-4c5e-b67c-2234a2d38e66");
8626        let uuid2 = uuid16_from_str("0826cc06-d2e3-4599-b4ad-af5fa6905cdb");
8627        let item_name = Field::LIST_FIELD_DEFAULT_NAME;
8628        let uf_tri = UnionFields::try_new(
8629            vec![0, 1, 2],
8630            vec![
8631                Field::new("int", DataType::Int32, false),
8632                Field::new("string", DataType::Utf8, false),
8633                Field::new("boolean", DataType::Boolean, false),
8634            ],
8635        )
8636        .unwrap();
8637        let uf_arr_items = UnionFields::try_new(
8638            vec![0, 1, 2],
8639            vec![
8640                Field::new("null", DataType::Null, false),
8641                Field::new("string", DataType::Utf8, false),
8642                Field::new("long", DataType::Int64, false),
8643            ],
8644        )
8645        .unwrap();
8646        let arr_items_field = Arc::new(Field::new(
8647            item_name,
8648            DataType::Union(uf_arr_items.clone(), UnionMode::Dense),
8649            true,
8650        ));
8651        let uf_map_vals = UnionFields::try_new(
8652            vec![0, 1, 2],
8653            vec![
8654                Field::new("string", DataType::Utf8, false),
8655                Field::new("double", DataType::Float64, false),
8656                Field::new("null", DataType::Null, false),
8657            ],
8658        )
8659        .unwrap();
8660        let map_entries_field = Arc::new(Field::new(
8661            "entries",
8662            DataType::Struct(Fields::from(vec![
8663                Field::new("key", DataType::Utf8, false),
8664                Field::new(
8665                    "value",
8666                    DataType::Union(uf_map_vals.clone(), UnionMode::Dense),
8667                    true,
8668                ),
8669            ])),
8670            false,
8671        ));
8672        // Enum metadata for Color (now includes name/namespace)
8673        let mut enum_md_color = {
8674            let mut m = HashMap::<String, String>::new();
8675            m.insert(
8676                crate::schema::AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
8677                serde_json::to_string(&vec!["RED", "GREEN", "BLUE"]).unwrap(),
8678            );
8679            m
8680        };
8681        enum_md_color.insert(AVRO_NAME_METADATA_KEY.to_string(), "Color".to_string());
8682        enum_md_color.insert(
8683            AVRO_NAMESPACE_METADATA_KEY.to_string(),
8684            "org.apache.arrow.avrotests.v1.types".to_string(),
8685        );
8686        let union_rec_a_fields = Fields::from(vec![
8687            Field::new("a", DataType::Int32, false),
8688            Field::new("b", DataType::Utf8, false),
8689        ]);
8690        let union_rec_b_fields = Fields::from(vec![
8691            Field::new("x", DataType::Int64, false),
8692            Field::new("y", DataType::Binary, false),
8693        ]);
8694        let union_map_entries = Arc::new(Field::new(
8695            "entries",
8696            DataType::Struct(Fields::from(vec![
8697                Field::new("key", DataType::Utf8, false),
8698                Field::new("value", DataType::Utf8, false),
8699            ])),
8700            false,
8701        ));
8702        let person_md = {
8703            let mut m = HashMap::<String, String>::new();
8704            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Person".to_string());
8705            m.insert(
8706                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8707                "com.example".to_string(),
8708            );
8709            m
8710        };
8711        let maybe_auth_md = {
8712            let mut m = HashMap::<String, String>::new();
8713            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "MaybeAuth".to_string());
8714            m.insert(
8715                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8716                "org.apache.arrow.avrotests.v1.types".to_string(),
8717            );
8718            m
8719        };
8720        let address_md = {
8721            let mut m = HashMap::<String, String>::new();
8722            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Address".to_string());
8723            m.insert(
8724                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8725                "org.apache.arrow.avrotests.v1.types".to_string(),
8726            );
8727            m
8728        };
8729        let rec_a_md = {
8730            let mut m = HashMap::<String, String>::new();
8731            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "RecA".to_string());
8732            m.insert(
8733                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8734                "org.apache.arrow.avrotests.v1.types".to_string(),
8735            );
8736            m
8737        };
8738        let rec_b_md = {
8739            let mut m = HashMap::<String, String>::new();
8740            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "RecB".to_string());
8741            m.insert(
8742                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8743                "org.apache.arrow.avrotests.v1.types".to_string(),
8744            );
8745            m
8746        };
8747        let uf_union_big = UnionFields::try_new(
8748            vec![0, 1, 2, 3, 4],
8749            vec![
8750                Field::new(
8751                    "map",
8752                    DataType::Map(union_map_entries.clone(), false),
8753                    false,
8754                ),
8755                Field::new(
8756                    "array",
8757                    DataType::List(Arc::new(Field::new(item_name, DataType::Int64, false))),
8758                    false,
8759                ),
8760                Field::new(
8761                    "org.apache.arrow.avrotests.v1.types.RecB",
8762                    DataType::Struct(union_rec_b_fields.clone()),
8763                    false,
8764                )
8765                .with_metadata(rec_b_md.clone()),
8766                Field::new(
8767                    "org.apache.arrow.avrotests.v1.types.RecA",
8768                    DataType::Struct(union_rec_a_fields.clone()),
8769                    false,
8770                )
8771                .with_metadata(rec_a_md.clone()),
8772                Field::new(
8773                    "org.apache.arrow.avrotests.v1.types.Color",
8774                    DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
8775                    false,
8776                )
8777                .with_metadata(enum_md_color.clone()),
8778            ],
8779        )
8780        .unwrap();
8781        let fx4_md = {
8782            let mut m = HashMap::<String, String>::new();
8783            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Fx4".to_string());
8784            m.insert(
8785                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8786                "org.apache.arrow.avrotests.v1".to_string(),
8787            );
8788            m
8789        };
8790        let uf_date_fixed4 = UnionFields::try_new(
8791            vec![0, 1],
8792            vec![
8793                Field::new(
8794                    "org.apache.arrow.avrotests.v1.Fx4",
8795                    DataType::FixedSizeBinary(4),
8796                    false,
8797                )
8798                .with_metadata(fx4_md.clone()),
8799                Field::new("date", DataType::Date32, false),
8800            ],
8801        )
8802        .unwrap();
8803        let dur12u_md = {
8804            let mut m = HashMap::<String, String>::new();
8805            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Dur12U".to_string());
8806            m.insert(
8807                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8808                "org.apache.arrow.avrotests.v1".to_string(),
8809            );
8810            m
8811        };
8812        let uf_dur_or_str = UnionFields::try_new(
8813            vec![0, 1],
8814            vec![
8815                Field::new("string", DataType::Utf8, false),
8816                Field::new(
8817                    "org.apache.arrow.avrotests.v1.Dur12U",
8818                    DataType::Interval(arrow_schema::IntervalUnit::MonthDayNano),
8819                    false,
8820                )
8821                .with_metadata(dur12u_md.clone()),
8822            ],
8823        )
8824        .unwrap();
8825        let fx10_md = {
8826            let mut m = HashMap::<String, String>::new();
8827            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Fx10".to_string());
8828            m.insert(
8829                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8830                "org.apache.arrow.avrotests.v1".to_string(),
8831            );
8832            m
8833        };
8834        let uf_uuid_or_fx10 = UnionFields::try_new(
8835            vec![0, 1],
8836            vec![
8837                Field::new(
8838                    "org.apache.arrow.avrotests.v1.Fx10",
8839                    DataType::FixedSizeBinary(10),
8840                    false,
8841                )
8842                .with_metadata(fx10_md.clone()),
8843                add_uuid_ext_union(Field::new("uuid", DataType::FixedSizeBinary(16), false)),
8844            ],
8845        )
8846        .unwrap();
8847        let uf_kv_val = UnionFields::try_new(
8848            vec![0, 1, 2],
8849            vec![
8850                Field::new("null", DataType::Null, false),
8851                Field::new("int", DataType::Int32, false),
8852                Field::new("long", DataType::Int64, false),
8853            ],
8854        )
8855        .unwrap();
8856        let kv_fields = Fields::from(vec![
8857            Field::new("key", DataType::Utf8, false),
8858            Field::new(
8859                "val",
8860                DataType::Union(uf_kv_val.clone(), UnionMode::Dense),
8861                true,
8862            ),
8863        ]);
8864        let kv_md = {
8865            let mut m = HashMap::<String, String>::new();
8866            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "KV".to_string());
8867            m.insert(
8868                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8869                "org.apache.arrow.avrotests.v1.types".to_string(),
8870            );
8871            m
8872        };
8873        let kv_item_field = Arc::new(
8874            Field::new(item_name, DataType::Struct(kv_fields.clone()), false).with_metadata(kv_md),
8875        );
8876        let map_int_entries = Arc::new(Field::new(
8877            "entries",
8878            DataType::Struct(Fields::from(vec![
8879                Field::new("key", DataType::Utf8, false),
8880                Field::new("value", DataType::Int32, false),
8881            ])),
8882            false,
8883        ));
8884        let uf_map_or_array = UnionFields::try_new(
8885            vec![0, 1],
8886            vec![
8887                Field::new(
8888                    "array",
8889                    DataType::List(Arc::new(Field::new(item_name, DataType::Int32, false))),
8890                    false,
8891                ),
8892                Field::new("map", DataType::Map(map_int_entries.clone(), false), false),
8893            ],
8894        )
8895        .unwrap();
8896        let mut enum_md_status = {
8897            let mut m = HashMap::<String, String>::new();
8898            m.insert(
8899                crate::schema::AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(),
8900                serde_json::to_string(&vec!["UNKNOWN", "NEW", "PROCESSING", "DONE"]).unwrap(),
8901            );
8902            m
8903        };
8904        enum_md_status.insert(AVRO_NAME_METADATA_KEY.to_string(), "Status".to_string());
8905        enum_md_status.insert(
8906            AVRO_NAMESPACE_METADATA_KEY.to_string(),
8907            "org.apache.arrow.avrotests.v1.types".to_string(),
8908        );
8909        let mut dec20_md = HashMap::<String, String>::new();
8910        dec20_md.insert("precision".to_string(), "20".to_string());
8911        dec20_md.insert("scale".to_string(), "4".to_string());
8912        dec20_md.insert(AVRO_NAME_METADATA_KEY.to_string(), "DecFix20".to_string());
8913        dec20_md.insert(
8914            AVRO_NAMESPACE_METADATA_KEY.to_string(),
8915            "org.apache.arrow.avrotests.v1.types".to_string(),
8916        );
8917        let mut dec10_md = HashMap::<String, String>::new();
8918        dec10_md.insert("precision".to_string(), "10".to_string());
8919        dec10_md.insert("scale".to_string(), "2".to_string());
8920        let fx16_top_md = {
8921            let mut m = HashMap::<String, String>::new();
8922            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Fx16".to_string());
8923            m.insert(
8924                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8925                "org.apache.arrow.avrotests.v1.types".to_string(),
8926            );
8927            m
8928        };
8929        let dur12_top_md = {
8930            let mut m = HashMap::<String, String>::new();
8931            m.insert(AVRO_NAME_METADATA_KEY.to_string(), "Dur12".to_string());
8932            m.insert(
8933                AVRO_NAMESPACE_METADATA_KEY.to_string(),
8934                "org.apache.arrow.avrotests.v1.types".to_string(),
8935            );
8936            m
8937        };
8938        #[cfg(feature = "small_decimals")]
8939        let dec20_dt = DataType::Decimal128(20, 4);
8940        #[cfg(not(feature = "small_decimals"))]
8941        let dec20_dt = DataType::Decimal128(20, 4);
8942        #[cfg(feature = "small_decimals")]
8943        let dec10_dt = DataType::Decimal64(10, 2);
8944        #[cfg(not(feature = "small_decimals"))]
8945        let dec10_dt = DataType::Decimal128(10, 2);
8946        let fields: Vec<FieldRef> = vec![
8947            Arc::new(
8948                Field::new(
8949                    "person",
8950                    DataType::Struct(Fields::from(vec![
8951                        Field::new("name", DataType::Utf8, false),
8952                        Field::new("age", DataType::Int32, false),
8953                    ])),
8954                    false,
8955                )
8956                .with_metadata(person_md),
8957            ),
8958            Arc::new(Field::new("old_count", DataType::Int32, false)),
8959            Arc::new(Field::new(
8960                "union_map_or_array_int",
8961                DataType::Union(uf_map_or_array.clone(), UnionMode::Dense),
8962                false,
8963            )),
8964            Arc::new(Field::new(
8965                "array_records_with_union",
8966                DataType::List(kv_item_field.clone()),
8967                false,
8968            )),
8969            Arc::new(Field::new(
8970                "union_uuid_or_fixed10",
8971                DataType::Union(uf_uuid_or_fx10.clone(), UnionMode::Dense),
8972                false,
8973            )),
8974            Arc::new(Field::new(
8975                "union_interval_or_string",
8976                DataType::Union(uf_dur_or_str.clone(), UnionMode::Dense),
8977                false,
8978            )),
8979            Arc::new(Field::new(
8980                "union_date_or_fixed4",
8981                DataType::Union(uf_date_fixed4.clone(), UnionMode::Dense),
8982                false,
8983            )),
8984            Arc::new(Field::new(
8985                "union_enum_record_array_map",
8986                DataType::Union(uf_union_big.clone(), UnionMode::Dense),
8987                false,
8988            )),
8989            Arc::new(
8990                Field::new(
8991                    "maybe_auth",
8992                    DataType::Struct(Fields::from(vec![
8993                        Field::new("user", DataType::Utf8, false),
8994                        Field::new("token", DataType::Binary, true), // [bytes,null] -> nullable bytes
8995                    ])),
8996                    false,
8997                )
8998                .with_metadata(maybe_auth_md),
8999            ),
9000            Arc::new(
9001                Field::new(
9002                    "address",
9003                    DataType::Struct(Fields::from(vec![
9004                        Field::new("street_name", DataType::Utf8, false),
9005                        Field::new("zip", DataType::Int32, false),
9006                        Field::new("country", DataType::Utf8, false),
9007                    ])),
9008                    false,
9009                )
9010                .with_metadata(address_md),
9011            ),
9012            Arc::new(Field::new(
9013                "map_union",
9014                DataType::Map(map_entries_field.clone(), false),
9015                false,
9016            )),
9017            Arc::new(Field::new(
9018                "arr_union",
9019                DataType::List(arr_items_field.clone()),
9020                false,
9021            )),
9022            Arc::new(
9023                Field::new(
9024                    "status",
9025                    DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
9026                    false,
9027                )
9028                .with_metadata(enum_md_status.clone()),
9029            ),
9030            Arc::new(
9031                Field::new(
9032                    "interval_mdn",
9033                    DataType::Interval(IntervalUnit::MonthDayNano),
9034                    false,
9035                )
9036                .with_metadata(dur12_top_md.clone()),
9037            ),
9038            Arc::new(Field::new(
9039                "ts_micros_local",
9040                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None),
9041                false,
9042            )),
9043            Arc::new(Field::new(
9044                "ts_millis_local",
9045                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, None),
9046                false,
9047            )),
9048            Arc::new(Field::new(
9049                "ts_micros_utc",
9050                DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, Some("+00:00".into())),
9051                false,
9052            )),
9053            Arc::new(Field::new(
9054                "ts_millis_utc",
9055                DataType::Timestamp(arrow_schema::TimeUnit::Millisecond, Some("+00:00".into())),
9056                false,
9057            )),
9058            Arc::new(Field::new(
9059                "t_micros",
9060                DataType::Time64(arrow_schema::TimeUnit::Microsecond),
9061                false,
9062            )),
9063            Arc::new(Field::new(
9064                "t_millis",
9065                DataType::Time32(arrow_schema::TimeUnit::Millisecond),
9066                false,
9067            )),
9068            Arc::new(Field::new("d_date", DataType::Date32, false)),
9069            Arc::new(add_uuid_ext_top(Field::new(
9070                "uuid_str",
9071                DataType::FixedSizeBinary(16),
9072                false,
9073            ))),
9074            Arc::new(Field::new("dec_fix_s20_4", dec20_dt, false).with_metadata(dec20_md.clone())),
9075            Arc::new(
9076                Field::new("dec_bytes_s10_2", dec10_dt, false).with_metadata(dec10_md.clone()),
9077            ),
9078            Arc::new(
9079                Field::new("fx16_plain", DataType::FixedSizeBinary(16), false)
9080                    .with_metadata(fx16_top_md.clone()),
9081            ),
9082            Arc::new(Field::new("raw_bytes", DataType::Binary, false)),
9083            Arc::new(Field::new("str_utf8", DataType::Utf8, false)),
9084            Arc::new(Field::new(
9085                "tri_union_prim",
9086                DataType::Union(uf_tri.clone(), UnionMode::Dense),
9087                false,
9088            )),
9089            Arc::new(Field::new("opt_str_nullsecond", DataType::Utf8, true)),
9090            Arc::new(Field::new("opt_i32_nullfirst", DataType::Int32, true)),
9091            Arc::new(Field::new("count_i64", DataType::Int64, false)),
9092            Arc::new(Field::new("count_i32", DataType::Int64, false)),
9093            Arc::new(Field::new("ratio_f64", DataType::Float64, false)),
9094            Arc::new(Field::new("ratio_f32", DataType::Float64, false)),
9095            Arc::new(Field::new("flag", DataType::Boolean, false)),
9096            Arc::new(Field::new("identifier", DataType::Int64, false)),
9097        ];
9098        let expected_schema = Arc::new(arrow_schema::Schema::new(Fields::from(fields)));
9099        let mut cols: Vec<ArrayRef> = vec![
9100            Arc::new(StructArray::new(
9101                match expected_schema
9102                    .field_with_name("person")
9103                    .unwrap()
9104                    .data_type()
9105                {
9106                    DataType::Struct(fs) => fs.clone(),
9107                    _ => unreachable!(),
9108                },
9109                vec![
9110                    Arc::new(StringArray::from(vec!["Alice", "Bob", "Carol", "Dave"])) as ArrayRef,
9111                    Arc::new(Int32Array::from(vec![30, 0, 25, 41])) as ArrayRef,
9112                ],
9113                None,
9114            )) as ArrayRef,
9115            Arc::new(Int32Array::from(vec![100, 42, 7, 42])) as ArrayRef,
9116        ];
9117        {
9118            let map_child: ArrayRef = {
9119                let keys = StringArray::from(vec!["x", "y", "only"]);
9120                let vals = Int32Array::from(vec![1, 2, 10]);
9121                let entries = StructArray::new(
9122                    Fields::from(vec![
9123                        Field::new("key", DataType::Utf8, false),
9124                        Field::new("value", DataType::Int32, false),
9125                    ]),
9126                    vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
9127                    None,
9128                );
9129                let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3]));
9130                Arc::new(MapArray::new(
9131                    map_int_entries.clone(),
9132                    moff,
9133                    entries,
9134                    None,
9135                    false,
9136                )) as ArrayRef
9137            };
9138            let list_child: ArrayRef = {
9139                let values = Int32Array::from(vec![1, 2, 3, 0]);
9140                let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4]));
9141                Arc::new(
9142                    ListArray::try_new(
9143                        Arc::new(Field::new(item_name, DataType::Int32, false)),
9144                        offsets,
9145                        Arc::new(values),
9146                        None,
9147                    )
9148                    .unwrap(),
9149                ) as ArrayRef
9150            };
9151            let tids = vec![1, 0, 1, 0];
9152            let offs = vec![0, 0, 1, 1];
9153            let arr = mk_dense_union(&uf_map_or_array, tids, offs, |f| match f.name().as_str() {
9154                "array" => Some(list_child.clone()),
9155                "map" => Some(map_child.clone()),
9156                _ => None,
9157            });
9158            cols.push(arr);
9159        }
9160        {
9161            let keys = Arc::new(StringArray::from(vec!["k1", "k2", "k", "k3", "x"])) as ArrayRef;
9162            let type_ids = vec![1, 0, 2, 0, 1];
9163            let offsets = vec![0, 0, 0, 1, 1];
9164            let vals = mk_dense_union(&uf_kv_val, type_ids, offsets, |f| match f.data_type() {
9165                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![5, -5])) as ArrayRef),
9166                DataType::Int64 => Some(Arc::new(Int64Array::from(vec![99i64])) as ArrayRef),
9167                DataType::Null => Some(Arc::new(NullArray::new(2)) as ArrayRef),
9168                _ => None,
9169            });
9170            let values_struct =
9171                Arc::new(StructArray::new(kv_fields.clone(), vec![keys, vals], None));
9172            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 2, 3, 4, 5]));
9173            let arr = Arc::new(
9174                ListArray::try_new(kv_item_field.clone(), list_offsets, values_struct, None)
9175                    .unwrap(),
9176            ) as ArrayRef;
9177            cols.push(arr);
9178        }
9179        {
9180            let type_ids = vec![1, 0, 1, 0]; // [uuid, fixed10, uuid, fixed10] but uf order = [fixed10, uuid]
9181            let offs = vec![0, 0, 1, 1];
9182            let arr = mk_dense_union(&uf_uuid_or_fx10, type_ids, offs, |f| match f.data_type() {
9183                DataType::FixedSizeBinary(16) => {
9184                    let it = [Some(uuid1), Some(uuid2)].into_iter();
9185                    Some(Arc::new(
9186                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
9187                    ) as ArrayRef)
9188                }
9189                DataType::FixedSizeBinary(10) => {
9190                    let fx10_a = [0xAAu8; 10];
9191                    let fx10_b = [0x00u8, 0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88, 0x99];
9192                    let it = [Some(fx10_a), Some(fx10_b)].into_iter();
9193                    Some(Arc::new(
9194                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 10).unwrap(),
9195                    ) as ArrayRef)
9196                }
9197                _ => None,
9198            });
9199            cols.push(arr);
9200        }
9201        {
9202            let type_ids = vec![1, 0, 1, 0]; // [duration, string, duration, string] but uf order = [string, duration]
9203            let offs = vec![0, 0, 1, 1];
9204            let arr = mk_dense_union(&uf_dur_or_str, type_ids, offs, |f| match f.data_type() {
9205                DataType::Interval(arrow_schema::IntervalUnit::MonthDayNano) => Some(Arc::new(
9206                    IntervalMonthDayNanoArray::from(vec![dur_small, dur_large]),
9207                )
9208                    as ArrayRef),
9209                DataType::Utf8 => Some(Arc::new(StringArray::from(vec![
9210                    "duration-as-text",
9211                    "iso-8601-period-P1Y",
9212                ])) as ArrayRef),
9213                _ => None,
9214            });
9215            cols.push(arr);
9216        }
9217        {
9218            let type_ids = vec![1, 0, 1, 0]; // [date, fixed, date, fixed] but uf order = [fixed, date]
9219            let offs = vec![0, 0, 1, 1];
9220            let arr = mk_dense_union(&uf_date_fixed4, type_ids, offs, |f| match f.data_type() {
9221                DataType::Date32 => Some(Arc::new(Date32Array::from(vec![date_a, 0])) as ArrayRef),
9222                DataType::FixedSizeBinary(4) => {
9223                    let it = [Some(*b"\x00\x11\x22\x33"), Some(*b"ABCD")].into_iter();
9224                    Some(Arc::new(
9225                        FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 4).unwrap(),
9226                    ) as ArrayRef)
9227                }
9228                _ => None,
9229            });
9230            cols.push(arr);
9231        }
9232        {
9233            let tids = vec![4, 3, 1, 0]; // uf order = [map(0), array(1), RecB(2), RecA(3), enum(4)]
9234            let offs = vec![0, 0, 0, 0];
9235            let arr = mk_dense_union(&uf_union_big, tids, offs, |f| match f.data_type() {
9236                DataType::Dictionary(_, _) => {
9237                    let keys = Int32Array::from(vec![0i32]);
9238                    let values =
9239                        Arc::new(StringArray::from(vec!["RED", "GREEN", "BLUE"])) as ArrayRef;
9240                    Some(
9241                        Arc::new(DictionaryArray::<Int32Type>::try_new(keys, values).unwrap())
9242                            as ArrayRef,
9243                    )
9244                }
9245                DataType::Struct(fs) if fs == &union_rec_a_fields => {
9246                    let a = Int32Array::from(vec![7]);
9247                    let b = StringArray::from(vec!["rec"]);
9248                    Some(Arc::new(StructArray::new(
9249                        fs.clone(),
9250                        vec![Arc::new(a) as ArrayRef, Arc::new(b) as ArrayRef],
9251                        None,
9252                    )) as ArrayRef)
9253                }
9254                DataType::List(_) => {
9255                    let values = Int64Array::from(vec![1i64, 2, 3]);
9256                    let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3]));
9257                    Some(Arc::new(
9258                        ListArray::try_new(
9259                            Arc::new(Field::new(item_name, DataType::Int64, false)),
9260                            offsets,
9261                            Arc::new(values),
9262                            None,
9263                        )
9264                        .unwrap(),
9265                    ) as ArrayRef)
9266                }
9267                DataType::Map(_, _) => {
9268                    let keys = StringArray::from(vec!["k"]);
9269                    let vals = StringArray::from(vec!["v"]);
9270                    let entries = StructArray::new(
9271                        Fields::from(vec![
9272                            Field::new("key", DataType::Utf8, false),
9273                            Field::new("value", DataType::Utf8, false),
9274                        ]),
9275                        vec![Arc::new(keys) as ArrayRef, Arc::new(vals) as ArrayRef],
9276                        None,
9277                    );
9278                    let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 1]));
9279                    Some(Arc::new(MapArray::new(
9280                        union_map_entries.clone(),
9281                        moff,
9282                        entries,
9283                        None,
9284                        false,
9285                    )) as ArrayRef)
9286                }
9287                _ => None,
9288            });
9289            cols.push(arr);
9290        }
9291        {
9292            let fs = match expected_schema
9293                .field_with_name("maybe_auth")
9294                .unwrap()
9295                .data_type()
9296            {
9297                DataType::Struct(fs) => fs.clone(),
9298                _ => unreachable!(),
9299            };
9300            let user =
9301                Arc::new(StringArray::from(vec!["alice", "bob", "carol", "dave"])) as ArrayRef;
9302            let token_values: Vec<Option<&[u8]>> = vec![
9303                None,
9304                Some(b"\x01\x02\x03".as_ref()),
9305                None,
9306                Some(b"".as_ref()),
9307            ];
9308            let token = Arc::new(BinaryArray::from(token_values)) as ArrayRef;
9309            cols.push(Arc::new(StructArray::new(fs, vec![user, token], None)) as ArrayRef);
9310        }
9311        {
9312            let fs = match expected_schema
9313                .field_with_name("address")
9314                .unwrap()
9315                .data_type()
9316            {
9317                DataType::Struct(fs) => fs.clone(),
9318                _ => unreachable!(),
9319            };
9320            let street = Arc::new(StringArray::from(vec![
9321                "100 Main",
9322                "",
9323                "42 Galaxy Way",
9324                "End Ave",
9325            ])) as ArrayRef;
9326            let zip = Arc::new(Int32Array::from(vec![12345, 0, 42424, 1])) as ArrayRef;
9327            let country = Arc::new(StringArray::from(vec!["US", "CA", "US", "GB"])) as ArrayRef;
9328            cols.push(Arc::new(StructArray::new(fs, vec![street, zip, country], None)) as ArrayRef);
9329        }
9330        {
9331            let keys = StringArray::from(vec!["a", "b", "c", "neg", "pi", "ok"]);
9332            let moff = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 3, 4, 4, 6]));
9333            let tid_s = 0; // string
9334            let tid_d = 1; // double
9335            let tid_n = 2; // null
9336            let type_ids = vec![tid_d, tid_n, tid_s, tid_d, tid_d, tid_s];
9337            let offsets = vec![0, 0, 0, 1, 2, 1];
9338            let pi_5dp = (std::f64::consts::PI * 100_000.0).trunc() / 100_000.0;
9339            let vals = mk_dense_union(&uf_map_vals, type_ids, offsets, |f| match f.data_type() {
9340                DataType::Float64 => {
9341                    Some(Arc::new(Float64Array::from(vec![1.5f64, -0.5, pi_5dp])) as ArrayRef)
9342                }
9343                DataType::Utf8 => {
9344                    Some(Arc::new(StringArray::from(vec!["yes", "true"])) as ArrayRef)
9345                }
9346                DataType::Null => Some(Arc::new(NullArray::new(1)) as ArrayRef),
9347                _ => None,
9348            });
9349            let entries = StructArray::new(
9350                Fields::from(vec![
9351                    Field::new("key", DataType::Utf8, false),
9352                    Field::new(
9353                        "value",
9354                        DataType::Union(uf_map_vals.clone(), UnionMode::Dense),
9355                        true,
9356                    ),
9357                ]),
9358                vec![Arc::new(keys) as ArrayRef, vals],
9359                None,
9360            );
9361            let map = Arc::new(MapArray::new(
9362                map_entries_field.clone(),
9363                moff,
9364                entries,
9365                None,
9366                false,
9367            )) as ArrayRef;
9368            cols.push(map);
9369        }
9370        {
9371            let type_ids = vec![
9372                2, 1, 0, 2, 0, 1, 2, 2, 1, 0,
9373                2, // long,string,null,long,null,string,long,long,string,null,long
9374            ];
9375            let offsets = vec![0, 0, 0, 1, 1, 1, 2, 3, 2, 2, 4];
9376            let values =
9377                mk_dense_union(&uf_arr_items, type_ids, offsets, |f| match f.data_type() {
9378                    DataType::Int64 => {
9379                        Some(Arc::new(Int64Array::from(vec![1i64, -3, 0, -1, 0])) as ArrayRef)
9380                    }
9381                    DataType::Utf8 => {
9382                        Some(Arc::new(StringArray::from(vec!["x", "z", "end"])) as ArrayRef)
9383                    }
9384                    DataType::Null => Some(Arc::new(NullArray::new(3)) as ArrayRef),
9385                    _ => None,
9386                });
9387            let list_offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0, 4, 7, 8, 11]));
9388            let arr = Arc::new(
9389                ListArray::try_new(arr_items_field.clone(), list_offsets, values, None).unwrap(),
9390            ) as ArrayRef;
9391            cols.push(arr);
9392        }
9393        {
9394            let keys = Int32Array::from(vec![1, 2, 3, 0]); // NEW, PROCESSING, DONE, UNKNOWN
9395            let values = Arc::new(StringArray::from(vec![
9396                "UNKNOWN",
9397                "NEW",
9398                "PROCESSING",
9399                "DONE",
9400            ])) as ArrayRef;
9401            let dict = DictionaryArray::<Int32Type>::try_new(keys, values).unwrap();
9402            cols.push(Arc::new(dict) as ArrayRef);
9403        }
9404        cols.push(Arc::new(IntervalMonthDayNanoArray::from(vec![
9405            dur_small, dur_zero, dur_large, dur_2years,
9406        ])) as ArrayRef);
9407        cols.push(Arc::new(TimestampMicrosecondArray::from(vec![
9408            ts_us_2024_01_01 + 123_456,
9409            0,
9410            ts_us_2024_01_01 + 101_112,
9411            987_654_321,
9412        ])) as ArrayRef);
9413        cols.push(Arc::new(TimestampMillisecondArray::from(vec![
9414            ts_ms_2024_01_01 + 86_400_000,
9415            0,
9416            ts_ms_2024_01_01 + 789,
9417            123_456_789,
9418        ])) as ArrayRef);
9419        {
9420            let a = TimestampMicrosecondArray::from(vec![
9421                ts_us_2024_01_01,
9422                1,
9423                ts_us_2024_01_01 + 456,
9424                0,
9425            ])
9426            .with_timezone("+00:00");
9427            cols.push(Arc::new(a) as ArrayRef);
9428        }
9429        {
9430            let a = TimestampMillisecondArray::from(vec![
9431                ts_ms_2024_01_01,
9432                -1,
9433                ts_ms_2024_01_01 + 123,
9434                0,
9435            ])
9436            .with_timezone("+00:00");
9437            cols.push(Arc::new(a) as ArrayRef);
9438        }
9439        cols.push(Arc::new(Time64MicrosecondArray::from(vec![
9440            time_us_eod,
9441            0,
9442            1,
9443            1_000_000,
9444        ])) as ArrayRef);
9445        cols.push(Arc::new(Time32MillisecondArray::from(vec![
9446            time_ms_a,
9447            0,
9448            1,
9449            86_400_000 - 1,
9450        ])) as ArrayRef);
9451        cols.push(Arc::new(Date32Array::from(vec![date_a, 0, 1, 365])) as ArrayRef);
9452        {
9453            let it = [Some(uuid1), Some(uuid2), Some(uuid1), Some(uuid2)].into_iter();
9454            cols.push(Arc::new(
9455                FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
9456            ) as ArrayRef);
9457        }
9458        {
9459            #[cfg(feature = "small_decimals")]
9460            let arr = Arc::new(
9461                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
9462                    .with_precision_and_scale(20, 4)
9463                    .unwrap(),
9464            ) as ArrayRef;
9465            #[cfg(not(feature = "small_decimals"))]
9466            let arr = Arc::new(
9467                Decimal128Array::from_iter_values([1_234_567_891_234i128, -420_000i128, 0, -1i128])
9468                    .with_precision_and_scale(20, 4)
9469                    .unwrap(),
9470            ) as ArrayRef;
9471            cols.push(arr);
9472        }
9473        {
9474            #[cfg(feature = "small_decimals")]
9475            let arr = Arc::new(
9476                Decimal64Array::from_iter_values([123456i64, -1, 0, 9_999_999_999i64])
9477                    .with_precision_and_scale(10, 2)
9478                    .unwrap(),
9479            ) as ArrayRef;
9480            #[cfg(not(feature = "small_decimals"))]
9481            let arr = Arc::new(
9482                Decimal128Array::from_iter_values([123456i128, -1, 0, 9_999_999_999i128])
9483                    .with_precision_and_scale(10, 2)
9484                    .unwrap(),
9485            ) as ArrayRef;
9486            cols.push(arr);
9487        }
9488        {
9489            let it = [
9490                Some(*b"0123456789ABCDEF"),
9491                Some([0u8; 16]),
9492                Some(*b"ABCDEFGHIJKLMNOP"),
9493                Some([0xAA; 16]),
9494            ]
9495            .into_iter();
9496            cols.push(Arc::new(
9497                FixedSizeBinaryArray::try_from_sparse_iter_with_size(it, 16).unwrap(),
9498            ) as ArrayRef);
9499        }
9500        cols.push(Arc::new(BinaryArray::from(vec![
9501            b"\x00\x01".as_ref(),
9502            b"".as_ref(),
9503            b"\xFF\x00".as_ref(),
9504            b"\x10\x20\x30\x40".as_ref(),
9505        ])) as ArrayRef);
9506        cols.push(Arc::new(StringArray::from(vec!["hello", "", "world", "✓ unicode"])) as ArrayRef);
9507        {
9508            let tids = vec![0, 1, 2, 1];
9509            let offs = vec![0, 0, 0, 1];
9510            let arr = mk_dense_union(&uf_tri, tids, offs, |f| match f.data_type() {
9511                DataType::Int32 => Some(Arc::new(Int32Array::from(vec![0])) as ArrayRef),
9512                DataType::Utf8 => Some(Arc::new(StringArray::from(vec!["hi", ""])) as ArrayRef),
9513                DataType::Boolean => Some(Arc::new(BooleanArray::from(vec![true])) as ArrayRef),
9514                _ => None,
9515            });
9516            cols.push(arr);
9517        }
9518        cols.push(Arc::new(StringArray::from(vec![
9519            Some("alpha"),
9520            None,
9521            Some("s3"),
9522            Some(""),
9523        ])) as ArrayRef);
9524        cols.push(Arc::new(Int32Array::from(vec![None, Some(42), None, Some(0)])) as ArrayRef);
9525        cols.push(Arc::new(Int64Array::from(vec![
9526            7_000_000_000i64,
9527            -2,
9528            0,
9529            -9_876_543_210i64,
9530        ])) as ArrayRef);
9531        cols.push(Arc::new(Int64Array::from(vec![7i64, -1, 0, 123])) as ArrayRef);
9532        cols.push(Arc::new(Float64Array::from(vec![2.5f64, -1.0, 7.0, -2.25])) as ArrayRef);
9533        cols.push(Arc::new(Float64Array::from(vec![1.25f64, -0.0, 3.5, 9.75])) as ArrayRef);
9534        cols.push(Arc::new(BooleanArray::from(vec![true, false, true, false])) as ArrayRef);
9535        cols.push(Arc::new(Int64Array::from(vec![1, 2, 3, 4])) as ArrayRef);
9536        let expected = RecordBatch::try_new(expected_schema, cols).unwrap();
9537        assert_eq!(
9538            expected, batch,
9539            "entire RecordBatch mismatch (schema, all columns, all rows)"
9540        );
9541    }
9542}