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