/Users/andrewlamb/Software/arrow-rs/arrow-avro/src/reader/mod.rs
Line | Count | Source |
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 | | //! This module provides facilities to read Apache Avro-encoded files or streams |
21 | | //! into Arrow's `RecordBatch` format. In particular, it introduces: |
22 | | //! |
23 | | //! * `ReaderBuilder`: Configures Avro reading, e.g., batch size |
24 | | //! * `Reader`: Yields `RecordBatch` values, implementing `Iterator` |
25 | | //! * `Decoder`: A low-level push-based decoder for Avro records |
26 | | //! |
27 | | //! # Basic Usage |
28 | | //! |
29 | | //! `Reader` can be used directly with synchronous data sources, such as [`std::fs::File`]. |
30 | | //! |
31 | | //! ## Reading a Single Batch |
32 | | //! |
33 | | //! ``` |
34 | | //! # use std::fs::File; |
35 | | //! # use std::io::BufReader; |
36 | | //! # use arrow_avro::reader::ReaderBuilder; |
37 | | //! # let path = "avro/alltypes_plain.avro"; |
38 | | //! # let path = match std::env::var("ARROW_TEST_DATA") { |
39 | | //! # Ok(dir) => format!("{dir}/{path}"), |
40 | | //! # Err(_) => format!("../testing/data/{path}") |
41 | | //! # }; |
42 | | //! let file = File::open(path).unwrap(); |
43 | | //! let mut avro = ReaderBuilder::new().build(BufReader::new(file)).unwrap(); |
44 | | //! let batch = avro.next().unwrap(); |
45 | | //! ``` |
46 | | //! |
47 | | //! # Async Usage |
48 | | //! |
49 | | //! The lower-level `Decoder` can be integrated with various forms of async data streams, |
50 | | //! and is designed to be agnostic to different async IO primitives within |
51 | | //! the Rust ecosystem. It works by incrementally decoding Avro data from byte slices. |
52 | | //! |
53 | | //! For example, see below for how it could be used with an arbitrary `Stream` of `Bytes`: |
54 | | //! |
55 | | //! ``` |
56 | | //! # use std::task::{Poll, ready}; |
57 | | //! # use bytes::{Buf, Bytes}; |
58 | | //! # use arrow_schema::ArrowError; |
59 | | //! # use futures::stream::{Stream, StreamExt}; |
60 | | //! # use arrow_array::RecordBatch; |
61 | | //! # use arrow_avro::reader::Decoder; |
62 | | //! |
63 | | //! fn decode_stream<S: Stream<Item = Bytes> + Unpin>( |
64 | | //! mut decoder: Decoder, |
65 | | //! mut input: S, |
66 | | //! ) -> impl Stream<Item = Result<RecordBatch, ArrowError>> { |
67 | | //! let mut buffered = Bytes::new(); |
68 | | //! futures::stream::poll_fn(move |cx| { |
69 | | //! loop { |
70 | | //! if buffered.is_empty() { |
71 | | //! buffered = match ready!(input.poll_next_unpin(cx)) { |
72 | | //! Some(b) => b, |
73 | | //! None => break, |
74 | | //! }; |
75 | | //! } |
76 | | //! let decoded = match decoder.decode(buffered.as_ref()) { |
77 | | //! Ok(decoded) => decoded, |
78 | | //! Err(e) => return Poll::Ready(Some(Err(e))), |
79 | | //! }; |
80 | | //! let read = buffered.len(); |
81 | | //! buffered.advance(decoded); |
82 | | //! if decoded != read { |
83 | | //! break |
84 | | //! } |
85 | | //! } |
86 | | //! // Convert any fully-decoded rows to a RecordBatch, if available |
87 | | //! Poll::Ready(decoder.flush().transpose()) |
88 | | //! }) |
89 | | //! } |
90 | | //! ``` |
91 | | //! |
92 | | use crate::codec::{AvroField, AvroFieldBuilder}; |
93 | | use crate::schema::{ |
94 | | compare_schemas, generate_fingerprint, AvroSchema, Fingerprint, FingerprintAlgorithm, Schema, |
95 | | SchemaStore, SINGLE_OBJECT_MAGIC, |
96 | | }; |
97 | | use arrow_array::{Array, RecordBatch, RecordBatchReader}; |
98 | | use arrow_schema::{ArrowError, SchemaRef}; |
99 | | use block::BlockDecoder; |
100 | | use header::{Header, HeaderDecoder}; |
101 | | use indexmap::IndexMap; |
102 | | use record::RecordDecoder; |
103 | | use std::collections::HashMap; |
104 | | use std::io::BufRead; |
105 | | |
106 | | mod block; |
107 | | mod cursor; |
108 | | mod header; |
109 | | mod record; |
110 | | mod vlq; |
111 | | |
112 | | /// Read the Avro file header (magic, metadata, sync marker) from `reader`. |
113 | 96 | fn read_header<R: BufRead>(mut reader: R) -> Result<Header, ArrowError> { |
114 | 96 | let mut decoder = HeaderDecoder::default(); |
115 | | loop { |
116 | 104 | let buf = reader.fill_buf()?0 ; |
117 | 104 | if buf.is_empty() { |
118 | 0 | break; |
119 | 104 | } |
120 | 104 | let read = buf.len(); |
121 | 104 | let decoded = decoder.decode(buf)?0 ; |
122 | 104 | reader.consume(decoded); |
123 | 104 | if decoded != read { |
124 | 96 | break; |
125 | 8 | } |
126 | | } |
127 | 96 | decoder.flush().ok_or_else(|| {0 |
128 | 0 | ArrowError::ParseError("Unexpected EOF while reading Avro header".to_string()) |
129 | 0 | }) |
130 | 96 | } |
131 | | |
132 | | // NOTE: The Current ` is_incomplete_data ` below is temporary and will be improved prior to public release |
133 | 0 | fn is_incomplete_data(err: &ArrowError) -> bool { |
134 | 0 | matches!( |
135 | 0 | err, |
136 | 0 | ArrowError::ParseError(msg) |
137 | 0 | if msg.contains("Unexpected EOF") |
138 | | ) |
139 | 0 | } |
140 | | |
141 | | /// A low-level interface for decoding Avro-encoded bytes into Arrow `RecordBatch`. |
142 | | #[derive(Debug)] |
143 | | pub struct Decoder { |
144 | | active_decoder: RecordDecoder, |
145 | | active_fingerprint: Option<Fingerprint>, |
146 | | batch_size: usize, |
147 | | remaining_capacity: usize, |
148 | | cache: IndexMap<Fingerprint, RecordDecoder>, |
149 | | fingerprint_algorithm: FingerprintAlgorithm, |
150 | | utf8_view: bool, |
151 | | strict_mode: bool, |
152 | | pending_schema: Option<(Fingerprint, RecordDecoder)>, |
153 | | awaiting_body: bool, |
154 | | } |
155 | | |
156 | | impl Decoder { |
157 | | /// Return the Arrow schema for the rows decoded by this decoder |
158 | 73 | pub fn schema(&self) -> SchemaRef { |
159 | 73 | self.active_decoder.schema().clone() |
160 | 73 | } |
161 | | |
162 | | /// Return the configured maximum number of rows per batch |
163 | 0 | pub fn batch_size(&self) -> usize { |
164 | 0 | self.batch_size |
165 | 0 | } |
166 | | |
167 | | /// Feed `data` into the decoder row by row until we either: |
168 | | /// - consume all bytes in `data`, or |
169 | | /// - reach `batch_size` decoded rows. |
170 | | /// |
171 | | /// Returns the number of bytes consumed. |
172 | 10 | pub fn decode(&mut self, data: &[u8]) -> Result<usize, ArrowError> { |
173 | 10 | let mut total_consumed = 0usize; |
174 | 28 | while total_consumed < data.len() && self.remaining_capacity > 020 { |
175 | 19 | if self.awaiting_body { |
176 | 9 | match self.active_decoder.decode(&data[total_consumed..], 1) { |
177 | 9 | Ok(n) => { |
178 | 9 | self.remaining_capacity -= 1; |
179 | 9 | total_consumed += n; |
180 | 9 | self.awaiting_body = false; |
181 | 9 | continue; |
182 | | } |
183 | 0 | Err(ref e) if is_incomplete_data(e) => break, |
184 | 0 | err => return err, |
185 | | }; |
186 | 10 | } |
187 | 10 | match self.handle_prefix(&data[total_consumed..])?1 { |
188 | 0 | Some(0) => break, // insufficient bytes |
189 | 9 | Some(n) => { |
190 | 9 | total_consumed += n; |
191 | 9 | self.apply_pending_schema_if_batch_empty(); |
192 | 9 | self.awaiting_body = true; |
193 | 9 | } |
194 | | None => { |
195 | 0 | return Err(ArrowError::ParseError( |
196 | 0 | "Missing magic bytes and fingerprint".to_string(), |
197 | 0 | )) |
198 | | } |
199 | | } |
200 | | } |
201 | 9 | Ok(total_consumed) |
202 | 10 | } |
203 | | |
204 | | // Attempt to handle a single‑object‑encoding prefix at the current position. |
205 | | // |
206 | | // * Ok(None) – buffer does not start with the prefix. |
207 | | // * Ok(Some(0)) – prefix detected, but the buffer is too short; caller should await more bytes. |
208 | | // * Ok(Some(n)) – consumed `n > 0` bytes of a complete prefix (magic and fingerprint). |
209 | 14 | fn handle_prefix(&mut self, buf: &[u8]) -> Result<Option<usize>, ArrowError> { |
210 | | // Need at least the magic bytes to decide (2 bytes). |
211 | 14 | let Some(magic_bytes13 ) = buf.get(..SINGLE_OBJECT_MAGIC.len()) else { |
212 | 1 | return Ok(Some(0)); // Get more bytes |
213 | | }; |
214 | | // Bail out early if the magic does not match. |
215 | 13 | if magic_bytes != SINGLE_OBJECT_MAGIC { |
216 | 1 | return Ok(None); // Continue to decode the next record |
217 | 12 | } |
218 | | // Try to parse the fingerprint that follows the magic. |
219 | 12 | let fingerprint_size11 = match self.fingerprint_algorithm { |
220 | 12 | FingerprintAlgorithm::Rabin => self |
221 | 12 | .handle_fingerprint(&buf[SINGLE_OBJECT_MAGIC.len()..], |bytes| {11 |
222 | 11 | Fingerprint::Rabin(u64::from_le_bytes(bytes)) |
223 | 11 | })?1 , |
224 | | }; |
225 | | // Convert the inner result into a “bytes consumed” count. |
226 | | // NOTE: Incomplete fingerprint consumes no bytes. |
227 | 11 | let consumed = fingerprint_size.map_or(0, |n| n10 + SINGLE_OBJECT_MAGIC10 .len10 ()); |
228 | 11 | Ok(Some(consumed)) |
229 | 14 | } |
230 | | |
231 | | // Attempts to read and install a new fingerprint of `N` bytes. |
232 | | // |
233 | | // * Ok(None) – insufficient bytes (`buf.len() < `N`). |
234 | | // * Ok(Some(N)) – fingerprint consumed (always `N`). |
235 | 12 | fn handle_fingerprint<const N: usize>( |
236 | 12 | &mut self, |
237 | 12 | buf: &[u8], |
238 | 12 | fingerprint_from: impl FnOnce([u8; N]) -> Fingerprint, |
239 | 12 | ) -> Result<Option<usize>, ArrowError> { |
240 | | // Need enough bytes to get fingerprint (next N bytes) |
241 | 12 | let Some(fingerprint_bytes11 ) = buf.get(..N) else { |
242 | 1 | return Ok(None); // Insufficient bytes |
243 | | }; |
244 | | // SAFETY: length checked above. |
245 | 11 | let new_fingerprint = fingerprint_from(fingerprint_bytes.try_into().unwrap()); |
246 | | // If the fingerprint indicates a schema change, prepare to switch decoders. |
247 | 11 | if self.active_fingerprint != Some(new_fingerprint) { |
248 | 3 | let Some(new_decoder2 ) = self.cache.shift_remove(&new_fingerprint) else { |
249 | 1 | return Err(ArrowError::ParseError(format!( |
250 | 1 | "Unknown fingerprint: {new_fingerprint:?}" |
251 | 1 | ))); |
252 | | }; |
253 | 2 | self.pending_schema = Some((new_fingerprint, new_decoder)); |
254 | | // If there are already decoded rows, we must flush them first. |
255 | | // Reducing `remaining_capacity` to 0 ensures `flush` is called next. |
256 | 2 | if self.remaining_capacity < self.batch_size { |
257 | 0 | self.remaining_capacity = 0; |
258 | 2 | } |
259 | 8 | } |
260 | 10 | Ok(Some(N)) |
261 | 12 | } |
262 | | |
263 | 15 | fn apply_pending_schema(&mut self) { |
264 | 15 | if let Some((new_fingerprint1 , new_decoder1 )) = self.pending_schema.take() { |
265 | 1 | if let Some(old_fingerprint) = self.active_fingerprint.replace(new_fingerprint) { |
266 | 1 | let old_decoder = std::mem::replace(&mut self.active_decoder, new_decoder); |
267 | 1 | self.cache.shift_remove(&old_fingerprint); |
268 | 1 | self.cache.insert(old_fingerprint, old_decoder); |
269 | 1 | } else { |
270 | 0 | self.active_decoder = new_decoder; |
271 | 0 | } |
272 | 14 | } |
273 | 15 | } |
274 | | |
275 | 9 | fn apply_pending_schema_if_batch_empty(&mut self) { |
276 | 9 | if self.batch_is_empty() { |
277 | 7 | self.apply_pending_schema(); |
278 | 7 | }2 |
279 | 9 | } |
280 | | |
281 | 207 | fn flush_and_reset(&mut self) -> Result<Option<RecordBatch>, ArrowError> { |
282 | 207 | if self.batch_is_empty() { |
283 | 74 | return Ok(None); |
284 | 133 | } |
285 | 133 | let batch = self.active_decoder.flush()?0 ; |
286 | 133 | self.remaining_capacity = self.batch_size; |
287 | 133 | Ok(Some(batch)) |
288 | 207 | } |
289 | | |
290 | | /// Produce a `RecordBatch` if at least one row is fully decoded, returning |
291 | | /// `Ok(None)` if no new rows are available. |
292 | 8 | pub fn flush(&mut self) -> Result<Option<RecordBatch>, ArrowError> { |
293 | | // We must flush the active decoder before switching to the pending one. |
294 | 8 | let batch = self.flush_and_reset(); |
295 | 8 | self.apply_pending_schema(); |
296 | 8 | batch |
297 | 8 | } |
298 | | |
299 | | /// Returns the number of rows that can be added to this decoder before it is full. |
300 | 0 | pub fn capacity(&self) -> usize { |
301 | 0 | self.remaining_capacity |
302 | 0 | } |
303 | | |
304 | | /// Returns true if the decoder has reached its capacity for the current batch. |
305 | 274 | pub fn batch_is_full(&self) -> bool { |
306 | 274 | self.remaining_capacity == 0 |
307 | 274 | } |
308 | | |
309 | | /// Returns true if the decoder has not decoded any batches yet. |
310 | 216 | pub fn batch_is_empty(&self) -> bool { |
311 | 216 | self.remaining_capacity == self.batch_size |
312 | 216 | } |
313 | | |
314 | | // Decode either the block count or remaining capacity from `data` (an OCF block payload). |
315 | | // |
316 | | // Returns the number of bytes consumed from `data` along with the number of records decoded. |
317 | 126 | fn decode_block(&mut self, data: &[u8], count: usize) -> Result<(usize, usize), ArrowError> { |
318 | | // OCF decoding never interleaves records across blocks, so no chunking. |
319 | 126 | let to_decode = std::cmp::min(count, self.remaining_capacity); |
320 | 126 | if to_decode == 0 { |
321 | 0 | return Ok((0, 0)); |
322 | 126 | } |
323 | 126 | let consumed = self.active_decoder.decode(data, to_decode)?0 ; |
324 | 126 | self.remaining_capacity -= to_decode; |
325 | 126 | Ok((consumed, to_decode)) |
326 | 126 | } |
327 | | |
328 | | // Produce a `RecordBatch` if at least one row is fully decoded, returning |
329 | | // `Ok(None)` if no new rows are available. |
330 | 199 | fn flush_block(&mut self) -> Result<Option<RecordBatch>, ArrowError> { |
331 | 199 | self.flush_and_reset() |
332 | 199 | } |
333 | | } |
334 | | |
335 | | /// A builder to create an [`Avro Reader`](Reader) that reads Avro data |
336 | | /// into Arrow `RecordBatch`. |
337 | | #[derive(Debug)] |
338 | | pub struct ReaderBuilder { |
339 | | batch_size: usize, |
340 | | strict_mode: bool, |
341 | | utf8_view: bool, |
342 | | reader_schema: Option<AvroSchema>, |
343 | | writer_schema_store: Option<SchemaStore>, |
344 | | active_fingerprint: Option<Fingerprint>, |
345 | | } |
346 | | |
347 | | impl Default for ReaderBuilder { |
348 | 86 | fn default() -> Self { |
349 | 86 | Self { |
350 | 86 | batch_size: 1024, |
351 | 86 | strict_mode: false, |
352 | 86 | utf8_view: false, |
353 | 86 | reader_schema: None, |
354 | 86 | writer_schema_store: None, |
355 | 86 | active_fingerprint: None, |
356 | 86 | } |
357 | 86 | } |
358 | | } |
359 | | |
360 | | impl ReaderBuilder { |
361 | | /// Creates a new [`ReaderBuilder`] with default settings: |
362 | | /// - `batch_size` = 1024 |
363 | | /// - `strict_mode` = false |
364 | | /// - `utf8_view` = false |
365 | | /// - `reader_schema` = None |
366 | | /// - `writer_schema_store` = None |
367 | | /// - `active_fingerprint` = None |
368 | 86 | pub fn new() -> Self { |
369 | 86 | Self::default() |
370 | 86 | } |
371 | | |
372 | 92 | fn make_record_decoder( |
373 | 92 | &self, |
374 | 92 | writer_schema: &Schema, |
375 | 92 | reader_schema: Option<&Schema>, |
376 | 92 | ) -> Result<RecordDecoder, ArrowError> { |
377 | 92 | let mut builder = AvroFieldBuilder::new(writer_schema); |
378 | 92 | if let Some(reader_schema30 ) = reader_schema { |
379 | 30 | builder = builder.with_reader_schema(reader_schema); |
380 | 62 | } |
381 | 92 | let root89 = builder |
382 | 92 | .with_utf8view(self.utf8_view) |
383 | 92 | .with_strict_mode(self.strict_mode) |
384 | 92 | .build()?3 ; |
385 | 89 | RecordDecoder::try_new_with_options(root.data_type(), self.utf8_view) |
386 | 92 | } |
387 | | |
388 | 92 | fn make_record_decoder_from_schemas( |
389 | 92 | &self, |
390 | 92 | writer_schema: &Schema, |
391 | 92 | reader_schema: Option<&AvroSchema>, |
392 | 92 | ) -> Result<RecordDecoder, ArrowError> { |
393 | 92 | let reader_schema_raw = reader_schema.map(|s| s30 .schema30 ()).transpose()?0 ; |
394 | 92 | self.make_record_decoder(writer_schema, reader_schema_raw.as_ref()) |
395 | 92 | } |
396 | | |
397 | 83 | fn make_decoder_with_parts( |
398 | 83 | &self, |
399 | 83 | active_decoder: RecordDecoder, |
400 | 83 | active_fingerprint: Option<Fingerprint>, |
401 | 83 | cache: IndexMap<Fingerprint, RecordDecoder>, |
402 | 83 | fingerprint_algorithm: FingerprintAlgorithm, |
403 | 83 | ) -> Decoder { |
404 | 83 | Decoder { |
405 | 83 | batch_size: self.batch_size, |
406 | 83 | remaining_capacity: self.batch_size, |
407 | 83 | active_fingerprint, |
408 | 83 | active_decoder, |
409 | 83 | cache, |
410 | 83 | utf8_view: self.utf8_view, |
411 | 83 | fingerprint_algorithm, |
412 | 83 | strict_mode: self.strict_mode, |
413 | 83 | pending_schema: None, |
414 | 83 | awaiting_body: false, |
415 | 83 | } |
416 | 83 | } |
417 | | |
418 | 86 | fn make_decoder( |
419 | 86 | &self, |
420 | 86 | header: Option<&Header>, |
421 | 86 | reader_schema: Option<&AvroSchema>, |
422 | 86 | ) -> Result<Decoder, ArrowError> { |
423 | 86 | if let Some(hdr76 ) = header { |
424 | 76 | let writer_schema = hdr |
425 | 76 | .schema() |
426 | 76 | .map_err(|e| ArrowError::ExternalError(Box::new(e)0 ))?0 |
427 | 76 | .ok_or_else(|| {0 |
428 | 0 | ArrowError::ParseError("No Avro schema present in file header".into()) |
429 | 0 | })?; |
430 | 73 | let record_decoder = |
431 | 76 | self.make_record_decoder_from_schemas(&writer_schema, reader_schema)?3 ; |
432 | 73 | return Ok(self.make_decoder_with_parts( |
433 | 73 | record_decoder, |
434 | 73 | None, |
435 | 73 | IndexMap::new(), |
436 | 73 | FingerprintAlgorithm::Rabin, |
437 | 73 | )); |
438 | 10 | } |
439 | 10 | let store = self.writer_schema_store.as_ref().ok_or_else(|| {0 |
440 | 0 | ArrowError::ParseError("Writer schema store required for raw Avro".into()) |
441 | 0 | })?; |
442 | 10 | let fingerprints = store.fingerprints(); |
443 | 10 | if fingerprints.is_empty() { |
444 | 0 | return Err(ArrowError::ParseError( |
445 | 0 | "Writer schema store must contain at least one schema".into(), |
446 | 0 | )); |
447 | 10 | } |
448 | 10 | let start_fingerprint = self |
449 | 10 | .active_fingerprint |
450 | 10 | .or_else(|| fingerprints.first()0 .copied0 ()) |
451 | 10 | .ok_or_else(|| {0 |
452 | 0 | ArrowError::ParseError("Could not determine initial schema fingerprint".into()) |
453 | 0 | })?; |
454 | 10 | let mut cache = IndexMap::with_capacity(fingerprints.len().saturating_sub(1)); |
455 | 10 | let mut active_decoder: Option<RecordDecoder> = None; |
456 | 16 | for fingerprint in store10 .fingerprints10 () { |
457 | 16 | let avro_schema = match store.lookup(&fingerprint) { |
458 | 16 | Some(schema) => schema, |
459 | | None => { |
460 | 0 | return Err(ArrowError::ComputeError(format!( |
461 | 0 | "Fingerprint {fingerprint:?} not found in schema store", |
462 | 0 | ))); |
463 | | } |
464 | | }; |
465 | 16 | let writer_schema = avro_schema.schema()?0 ; |
466 | 16 | let record_decoder = |
467 | 16 | self.make_record_decoder_from_schemas(&writer_schema, reader_schema)?0 ; |
468 | 16 | if fingerprint == start_fingerprint { |
469 | 10 | active_decoder = Some(record_decoder); |
470 | 10 | } else { |
471 | 6 | cache.insert(fingerprint, record_decoder); |
472 | 6 | } |
473 | | } |
474 | 10 | let active_decoder = active_decoder.ok_or_else(|| {0 |
475 | 0 | ArrowError::ComputeError(format!( |
476 | 0 | "Initial fingerprint {start_fingerprint:?} not found in schema store" |
477 | 0 | )) |
478 | 0 | })?; |
479 | 10 | Ok(self.make_decoder_with_parts( |
480 | 10 | active_decoder, |
481 | 10 | Some(start_fingerprint), |
482 | 10 | cache, |
483 | 10 | store.fingerprint_algorithm(), |
484 | 10 | )) |
485 | 86 | } |
486 | | |
487 | | /// Sets the row-based batch size |
488 | 75 | pub fn with_batch_size(mut self, batch_size: usize) -> Self { |
489 | 75 | self.batch_size = batch_size; |
490 | 75 | self |
491 | 75 | } |
492 | | |
493 | | /// Set whether to use StringViewArray for string data |
494 | | /// |
495 | | /// When enabled, string data from Avro files will be loaded into |
496 | | /// Arrow's StringViewArray instead of the standard StringArray. |
497 | 65 | pub fn with_utf8_view(mut self, utf8_view: bool) -> Self { |
498 | 65 | self.utf8_view = utf8_view; |
499 | 65 | self |
500 | 65 | } |
501 | | |
502 | | /// Get whether StringViewArray is enabled for string data |
503 | 0 | pub fn use_utf8view(&self) -> bool { |
504 | 0 | self.utf8_view |
505 | 0 | } |
506 | | |
507 | | /// Controls whether certain Avro unions of the form `[T, "null"]` should produce an error. |
508 | 2 | pub fn with_strict_mode(mut self, strict_mode: bool) -> Self { |
509 | 2 | self.strict_mode = strict_mode; |
510 | 2 | self |
511 | 2 | } |
512 | | |
513 | | /// Sets the Avro reader schema. |
514 | | /// |
515 | | /// If a schema is not provided, the schema will be read from the Avro file header. |
516 | 25 | pub fn with_reader_schema(mut self, schema: AvroSchema) -> Self { |
517 | 25 | self.reader_schema = Some(schema); |
518 | 25 | self |
519 | 25 | } |
520 | | |
521 | | /// Sets the `SchemaStore` used for resolving writer schemas. |
522 | | /// |
523 | | /// This is necessary when decoding single-object encoded data that identifies |
524 | | /// schemas by a fingerprint. The store allows the decoder to look up the |
525 | | /// full writer schema from a fingerprint embedded in the data. |
526 | | /// |
527 | | /// Defaults to `None`. |
528 | 10 | pub fn with_writer_schema_store(mut self, store: SchemaStore) -> Self { |
529 | 10 | self.writer_schema_store = Some(store); |
530 | 10 | self |
531 | 10 | } |
532 | | |
533 | | /// Sets the initial schema fingerprint for decoding single-object encoded data. |
534 | | /// |
535 | | /// This is useful when the data stream does not begin with a schema definition |
536 | | /// or fingerprint, allowing the decoder to start with a known schema from the |
537 | | /// `SchemaStore`. |
538 | | /// |
539 | | /// Defaults to `None`. |
540 | 10 | pub fn with_active_fingerprint(mut self, fp: Fingerprint) -> Self { |
541 | 10 | self.active_fingerprint = Some(fp); |
542 | 10 | self |
543 | 10 | } |
544 | | |
545 | | /// Create a [`Reader`] from this builder and a `BufRead` |
546 | 76 | pub fn build<R: BufRead>(self, mut reader: R) -> Result<Reader<R>, ArrowError> { |
547 | 76 | let header = read_header(&mut reader)?0 ; |
548 | 76 | let decoder73 = self.make_decoder(Some(&header), self.reader_schema.as_ref())?3 ; |
549 | 73 | Ok(Reader { |
550 | 73 | reader, |
551 | 73 | header, |
552 | 73 | decoder, |
553 | 73 | block_decoder: BlockDecoder::default(), |
554 | 73 | block_data: Vec::new(), |
555 | 73 | block_count: 0, |
556 | 73 | block_cursor: 0, |
557 | 73 | finished: false, |
558 | 73 | }) |
559 | 76 | } |
560 | | |
561 | | /// Create a [`Decoder`] from this builder. |
562 | 10 | pub fn build_decoder(self) -> Result<Decoder, ArrowError> { |
563 | 10 | if self.writer_schema_store.is_none() { |
564 | 0 | return Err(ArrowError::InvalidArgumentError( |
565 | 0 | "Building a decoder requires a writer schema store".to_string(), |
566 | 0 | )); |
567 | 10 | } |
568 | 10 | self.make_decoder(None, self.reader_schema.as_ref()) |
569 | 10 | } |
570 | | } |
571 | | |
572 | | /// A high-level Avro `Reader` that reads container-file blocks |
573 | | /// and feeds them into a row-level [`Decoder`]. |
574 | | #[derive(Debug)] |
575 | | pub struct Reader<R: BufRead> { |
576 | | reader: R, |
577 | | header: Header, |
578 | | decoder: Decoder, |
579 | | block_decoder: BlockDecoder, |
580 | | block_data: Vec<u8>, |
581 | | block_count: usize, |
582 | | block_cursor: usize, |
583 | | finished: bool, |
584 | | } |
585 | | |
586 | | impl<R: BufRead> Reader<R> { |
587 | | /// Return the Arrow schema discovered from the Avro file header |
588 | 73 | pub fn schema(&self) -> SchemaRef { |
589 | 73 | self.decoder.schema() |
590 | 73 | } |
591 | | |
592 | | /// Return the Avro container-file header |
593 | 0 | pub fn avro_header(&self) -> &Header { |
594 | 0 | &self.header |
595 | 0 | } |
596 | | |
597 | | /// Reads the next [`RecordBatch`] from the Avro file or `Ok(None)` on EOF |
598 | 199 | fn read(&mut self) -> Result<Option<RecordBatch>, ArrowError> { |
599 | 325 | 'outer: while !self.finished && !self.decoder.batch_is_full()274 { |
600 | 272 | while self.block_cursor == self.block_data.len() { |
601 | 146 | let buf = self.reader.fill_buf()?0 ; |
602 | 146 | if buf.is_empty() { |
603 | 73 | self.finished = true; |
604 | 73 | break 'outer; |
605 | 73 | } |
606 | | // Try to decode another block from the buffered reader. |
607 | 73 | let consumed = self.block_decoder.decode(buf)?0 ; |
608 | 73 | self.reader.consume(consumed); |
609 | 73 | if let Some(block) = self.block_decoder.flush() { |
610 | | // Successfully decoded a block. |
611 | 73 | self.block_data = if let Some(ref codec60 ) = self.header.compression()?0 { |
612 | 60 | codec.decompress(&block.data)?0 |
613 | | } else { |
614 | 13 | block.data |
615 | | }; |
616 | 73 | self.block_count = block.count; |
617 | 73 | self.block_cursor = 0; |
618 | 0 | } else if consumed == 0 { |
619 | | // The block decoder made no progress on a non-empty buffer. |
620 | 0 | return Err(ArrowError::ParseError( |
621 | 0 | "Could not decode next Avro block from partial data".to_string(), |
622 | 0 | )); |
623 | 0 | } |
624 | | } |
625 | | // Decode as many rows as will fit in the current batch |
626 | 126 | if self.block_cursor < self.block_data.len() { |
627 | 126 | let (consumed, records_decoded) = self |
628 | 126 | .decoder |
629 | 126 | .decode_block(&self.block_data[self.block_cursor..], self.block_count)?0 ; |
630 | 126 | self.block_cursor += consumed; |
631 | 126 | self.block_count -= records_decoded; |
632 | 0 | } |
633 | | } |
634 | 199 | self.decoder.flush_block() |
635 | 199 | } |
636 | | } |
637 | | |
638 | | impl<R: BufRead> Iterator for Reader<R> { |
639 | | type Item = Result<RecordBatch, ArrowError>; |
640 | | |
641 | 199 | fn next(&mut self) -> Option<Self::Item> { |
642 | 199 | self.read().transpose() |
643 | 199 | } |
644 | | } |
645 | | |
646 | | impl<R: BufRead> RecordBatchReader for Reader<R> { |
647 | 0 | fn schema(&self) -> SchemaRef { |
648 | 0 | self.schema() |
649 | 0 | } |
650 | | } |
651 | | |
652 | | #[cfg(test)] |
653 | | mod test { |
654 | | use crate::codec::{AvroDataType, AvroField, AvroFieldBuilder, Codec}; |
655 | | use crate::compression::CompressionCodec; |
656 | | use crate::reader::record::RecordDecoder; |
657 | | use crate::reader::vlq::VLQDecoder; |
658 | | use crate::reader::{read_header, Decoder, Reader, ReaderBuilder}; |
659 | | use crate::schema::{ |
660 | | AvroSchema, Fingerprint, FingerprintAlgorithm, PrimitiveType, Schema as AvroRaw, |
661 | | SchemaStore, AVRO_ENUM_SYMBOLS_METADATA_KEY, SINGLE_OBJECT_MAGIC, |
662 | | }; |
663 | | use crate::test_util::arrow_test_data; |
664 | | use arrow::array::ArrayDataBuilder; |
665 | | use arrow_array::builder::{ |
666 | | ArrayBuilder, BooleanBuilder, Float32Builder, Float64Builder, Int32Builder, Int64Builder, |
667 | | ListBuilder, MapBuilder, StringBuilder, StructBuilder, |
668 | | }; |
669 | | use arrow_array::types::{Int32Type, IntervalMonthDayNanoType}; |
670 | | use arrow_array::*; |
671 | | use arrow_buffer::{Buffer, NullBuffer, OffsetBuffer, ScalarBuffer}; |
672 | | use arrow_schema::{ArrowError, DataType, Field, Fields, IntervalUnit, Schema}; |
673 | | use bytes::{Buf, BufMut, Bytes}; |
674 | | use futures::executor::block_on; |
675 | | use futures::{stream, Stream, StreamExt, TryStreamExt}; |
676 | | use serde_json::Value; |
677 | | use std::collections::HashMap; |
678 | | use std::fs; |
679 | | use std::fs::File; |
680 | | use std::io::{BufReader, Cursor, Read}; |
681 | | use std::sync::Arc; |
682 | | use std::task::{ready, Poll}; |
683 | | |
684 | 46 | fn read_file(path: &str, batch_size: usize, utf8_view: bool) -> RecordBatch { |
685 | 46 | let file = File::open(path).unwrap(); |
686 | 46 | let reader = ReaderBuilder::new() |
687 | 46 | .with_batch_size(batch_size) |
688 | 46 | .with_utf8_view(utf8_view) |
689 | 46 | .build(BufReader::new(file)) |
690 | 46 | .unwrap(); |
691 | 46 | let schema = reader.schema(); |
692 | 46 | let batches = reader.collect::<Result<Vec<_>, _>>().unwrap(); |
693 | 46 | arrow::compute::concat_batches(&schema, &batches).unwrap() |
694 | 46 | } |
695 | | |
696 | 2 | fn read_file_strict( |
697 | 2 | path: &str, |
698 | 2 | batch_size: usize, |
699 | 2 | utf8_view: bool, |
700 | 2 | ) -> Result<Reader<BufReader<File>>, ArrowError> { |
701 | 2 | let file = File::open(path)?0 ; |
702 | 2 | ReaderBuilder::new() |
703 | 2 | .with_batch_size(batch_size) |
704 | 2 | .with_utf8_view(utf8_view) |
705 | 2 | .with_strict_mode(true) |
706 | 2 | .build(BufReader::new(file)) |
707 | 2 | } |
708 | | |
709 | 2 | fn decode_stream<S: Stream<Item = Bytes> + Unpin>( |
710 | 2 | mut decoder: Decoder, |
711 | 2 | mut input: S, |
712 | 2 | ) -> impl Stream<Item = Result<RecordBatch, ArrowError>> { |
713 | 2 | async_stream::try_stream! { |
714 | | if let Some(data) = input.next().await { |
715 | | let consumed = decoder.decode(&data)?; |
716 | | if consumed < data.len() { |
717 | | Err(ArrowError::ParseError( |
718 | | "did not consume all bytes".to_string(), |
719 | | ))?; |
720 | | } |
721 | | } |
722 | | if let Some(batch) = decoder.flush()? { |
723 | | yield batch |
724 | | } |
725 | | } |
726 | 2 | } |
727 | | |
728 | 12 | fn make_record_schema(pt: PrimitiveType) -> AvroSchema { |
729 | 12 | let js = format!( |
730 | 12 | r#"{{"type":"record","name":"TestRecord","fields":[{{"name":"a","type":"{}"}}]}}"#, |
731 | 12 | pt.as_ref() |
732 | | ); |
733 | 12 | AvroSchema::new(js) |
734 | 12 | } |
735 | | |
736 | 5 | fn make_two_schema_store() -> ( |
737 | 5 | SchemaStore, |
738 | 5 | Fingerprint, |
739 | 5 | Fingerprint, |
740 | 5 | AvroSchema, |
741 | 5 | AvroSchema, |
742 | 5 | ) { |
743 | 5 | let schema_int = make_record_schema(PrimitiveType::Int); |
744 | 5 | let schema_long = make_record_schema(PrimitiveType::Long); |
745 | 5 | let mut store = SchemaStore::new(); |
746 | 5 | let fp_int = store |
747 | 5 | .register(schema_int.clone()) |
748 | 5 | .expect("register int schema"); |
749 | 5 | let fp_long = store |
750 | 5 | .register(schema_long.clone()) |
751 | 5 | .expect("register long schema"); |
752 | 5 | (store, fp_int, fp_long, schema_int, schema_long) |
753 | 5 | } |
754 | | |
755 | 10 | fn make_prefix(fp: Fingerprint) -> Vec<u8> { |
756 | 10 | match fp { |
757 | 10 | Fingerprint::Rabin(v) => { |
758 | 10 | let mut out = Vec::with_capacity(2 + 8); |
759 | 10 | out.extend_from_slice(&SINGLE_OBJECT_MAGIC); |
760 | 10 | out.extend_from_slice(&v.to_le_bytes()); |
761 | 10 | out |
762 | | } |
763 | | } |
764 | 10 | } |
765 | | |
766 | 5 | fn make_decoder(store: &SchemaStore, fp: Fingerprint, reader_schema: &AvroSchema) -> Decoder { |
767 | 5 | ReaderBuilder::new() |
768 | 5 | .with_batch_size(8) |
769 | 5 | .with_reader_schema(reader_schema.clone()) |
770 | 5 | .with_writer_schema_store(store.clone()) |
771 | 5 | .with_active_fingerprint(fp) |
772 | 5 | .build_decoder() |
773 | 5 | .expect("decoder") |
774 | 5 | } |
775 | | |
776 | 4 | fn make_value_schema(pt: PrimitiveType) -> AvroSchema { |
777 | 4 | let json_schema = format!( |
778 | 4 | r#"{{"type":"record","name":"S","fields":[{{"name":"v","type":"{}"}}]}}"#, |
779 | 4 | pt.as_ref() |
780 | | ); |
781 | 4 | AvroSchema::new(json_schema) |
782 | 4 | } |
783 | | |
784 | 7 | fn encode_zigzag(value: i64) -> Vec<u8> { |
785 | 7 | let mut n = ((value << 1) ^ (value >> 63)) as u64; |
786 | 7 | let mut out = Vec::new(); |
787 | | loop { |
788 | 10 | if (n & !0x7F) == 0 { |
789 | 7 | out.push(n as u8); |
790 | 7 | break; |
791 | 3 | } else { |
792 | 3 | out.push(((n & 0x7F) | 0x80) as u8); |
793 | 3 | n >>= 7; |
794 | 3 | } |
795 | | } |
796 | 7 | out |
797 | 7 | } |
798 | | |
799 | 7 | fn make_message(fp: Fingerprint, value: i64) -> Vec<u8> { |
800 | 7 | let mut msg = make_prefix(fp); |
801 | 7 | msg.extend_from_slice(&encode_zigzag(value)); |
802 | 7 | msg |
803 | 7 | } |
804 | | |
805 | 18 | fn load_writer_schema_json(path: &str) -> Value { |
806 | 18 | let file = File::open(path).unwrap(); |
807 | 18 | let header = super::read_header(BufReader::new(file)).unwrap(); |
808 | 18 | let schema = header.schema().unwrap().unwrap(); |
809 | 18 | serde_json::to_value(&schema).unwrap() |
810 | 18 | } |
811 | | |
812 | 16 | fn make_reader_schema_with_promotions( |
813 | 16 | path: &str, |
814 | 16 | promotions: &HashMap<&str, &str>, |
815 | 16 | ) -> AvroSchema { |
816 | 16 | let mut root = load_writer_schema_json(path); |
817 | 16 | assert_eq!(root["type"], "record", "writer schema must be a record"0 ); |
818 | 16 | let fields = root |
819 | 16 | .get_mut("fields") |
820 | 16 | .and_then(|f| f.as_array_mut()) |
821 | 16 | .expect("record has fields"); |
822 | 176 | for f in fields16 .iter_mut16 () { |
823 | 176 | let Some(name) = f.get("name").and_then(|n| n.as_str()) else { |
824 | 0 | continue; |
825 | | }; |
826 | 176 | if let Some(new_ty56 ) = promotions.get(name) { |
827 | 56 | let ty = f.get_mut("type").expect("field has a type"); |
828 | 56 | match ty { |
829 | 0 | Value::String(_) => { |
830 | 0 | *ty = Value::String((*new_ty).to_string()); |
831 | 0 | } |
832 | | // Union |
833 | 56 | Value::Array(arr) => { |
834 | 56 | for b in arr.iter_mut() { |
835 | 56 | match b { |
836 | 56 | Value::String(s) if s != "null" => { |
837 | 56 | *b = Value::String((*new_ty).to_string()); |
838 | 56 | break; |
839 | | } |
840 | | Value::Object(_) => { |
841 | 0 | *b = Value::String((*new_ty).to_string()); |
842 | 0 | break; |
843 | | } |
844 | 0 | _ => {} |
845 | | } |
846 | | } |
847 | | } |
848 | 0 | Value::Object(_) => { |
849 | 0 | *ty = Value::String((*new_ty).to_string()); |
850 | 0 | } |
851 | 0 | _ => {} |
852 | | } |
853 | 120 | } |
854 | | } |
855 | 16 | AvroSchema::new(root.to_string()) |
856 | 16 | } |
857 | | |
858 | 17 | fn read_alltypes_with_reader_schema(path: &str, reader_schema: AvroSchema) -> RecordBatch { |
859 | 17 | let file = File::open(path).unwrap(); |
860 | 17 | let reader = ReaderBuilder::new() |
861 | 17 | .with_batch_size(1024) |
862 | 17 | .with_utf8_view(false) |
863 | 17 | .with_reader_schema(reader_schema) |
864 | 17 | .build(BufReader::new(file)) |
865 | 17 | .unwrap(); |
866 | 17 | let schema = reader.schema(); |
867 | 17 | let batches = reader.collect::<Result<Vec<_>, _>>().unwrap(); |
868 | 17 | arrow::compute::concat_batches(&schema, &batches).unwrap() |
869 | 17 | } |
870 | | |
871 | 2 | fn make_reader_schema_with_selected_fields_in_order( |
872 | 2 | path: &str, |
873 | 2 | selected: &[&str], |
874 | 2 | ) -> AvroSchema { |
875 | 2 | let mut root = load_writer_schema_json(path); |
876 | 2 | assert_eq!(root["type"], "record", "writer schema must be a record"0 ); |
877 | 2 | let writer_fields = root |
878 | 2 | .get("fields") |
879 | 2 | .and_then(|f| f.as_array()) |
880 | 2 | .expect("record has fields"); |
881 | 2 | let mut field_map: HashMap<String, Value> = HashMap::with_capacity(writer_fields.len()); |
882 | 24 | for f22 in writer_fields { |
883 | 22 | if let Some(name) = f.get("name").and_then(|n| n.as_str()) { |
884 | 22 | field_map.insert(name.to_string(), f.clone()); |
885 | 22 | }0 |
886 | | } |
887 | 2 | let mut new_fields = Vec::with_capacity(selected.len()); |
888 | 5 | for name3 in selected { |
889 | 3 | let f = field_map |
890 | 3 | .get(*name) |
891 | 3 | .unwrap_or_else(|| panic!("field '{name}' not found in writer schema"0 )) |
892 | 3 | .clone(); |
893 | 3 | new_fields.push(f); |
894 | | } |
895 | 2 | root["fields"] = Value::Array(new_fields); |
896 | 2 | AvroSchema::new(root.to_string()) |
897 | 2 | } |
898 | | |
899 | | #[test] |
900 | 1 | fn test_alltypes_schema_promotion_mixed() { |
901 | 1 | let files = [ |
902 | 1 | "avro/alltypes_plain.avro", |
903 | 1 | "avro/alltypes_plain.snappy.avro", |
904 | 1 | "avro/alltypes_plain.zstandard.avro", |
905 | 1 | "avro/alltypes_plain.bzip2.avro", |
906 | 1 | "avro/alltypes_plain.xz.avro", |
907 | 1 | ]; |
908 | 6 | for file5 in files { |
909 | 5 | let file = arrow_test_data(file); |
910 | 5 | let mut promotions: HashMap<&str, &str> = HashMap::new(); |
911 | 5 | promotions.insert("id", "long"); |
912 | 5 | promotions.insert("tinyint_col", "float"); |
913 | 5 | promotions.insert("smallint_col", "double"); |
914 | 5 | promotions.insert("int_col", "double"); |
915 | 5 | promotions.insert("bigint_col", "double"); |
916 | 5 | promotions.insert("float_col", "double"); |
917 | 5 | promotions.insert("date_string_col", "string"); |
918 | 5 | promotions.insert("string_col", "string"); |
919 | 5 | let reader_schema = make_reader_schema_with_promotions(&file, &promotions); |
920 | 5 | let batch = read_alltypes_with_reader_schema(&file, reader_schema); |
921 | 5 | let expected = RecordBatch::try_from_iter_with_nullable([ |
922 | 5 | ( |
923 | 5 | "id", |
924 | 5 | Arc::new(Int64Array::from(vec![4i64, 5, 6, 7, 2, 3, 0, 1])) as _, |
925 | 5 | true, |
926 | 5 | ), |
927 | | ( |
928 | 5 | "bool_col", |
929 | 40 | Arc::new5 (BooleanArray::from_iter5 ((0..8)5 .map5 (|x| Some(x % 2 == 0)))) as _5 , |
930 | | true, |
931 | | ), |
932 | | ( |
933 | 5 | "tinyint_col", |
934 | 5 | Arc::new(Float32Array::from_iter_values( |
935 | 40 | (0..8)5 .map5 (|x| (x % 2) as f32), |
936 | 5 | )) as _, |
937 | | true, |
938 | | ), |
939 | | ( |
940 | 5 | "smallint_col", |
941 | 5 | Arc::new(Float64Array::from_iter_values( |
942 | 40 | (0..8)5 .map5 (|x| (x % 2) as f64), |
943 | 5 | )) as _, |
944 | | true, |
945 | | ), |
946 | | ( |
947 | 5 | "int_col", |
948 | 5 | Arc::new(Float64Array::from_iter_values( |
949 | 40 | (0..8)5 .map5 (|x| (x % 2) as f64), |
950 | 5 | )) as _, |
951 | | true, |
952 | | ), |
953 | | ( |
954 | 5 | "bigint_col", |
955 | 5 | Arc::new(Float64Array::from_iter_values( |
956 | 40 | (0..8)5 .map5 (|x| ((x % 2) * 10) as f64), |
957 | 5 | )) as _, |
958 | | true, |
959 | | ), |
960 | | ( |
961 | 5 | "float_col", |
962 | 5 | Arc::new(Float64Array::from_iter_values( |
963 | 40 | (0..8)5 .map5 (|x| ((x % 2) as f32 * 1.1f32) as f64), |
964 | 5 | )) as _, |
965 | | true, |
966 | | ), |
967 | | ( |
968 | 5 | "double_col", |
969 | 5 | Arc::new(Float64Array::from_iter_values( |
970 | 40 | (0..8)5 .map5 (|x| (x % 2) as f64 * 10.1), |
971 | 5 | )) as _, |
972 | | true, |
973 | | ), |
974 | 5 | ( |
975 | 5 | "date_string_col", |
976 | 5 | Arc::new(StringArray::from(vec![ |
977 | 5 | "03/01/09", "03/01/09", "04/01/09", "04/01/09", "02/01/09", "02/01/09", |
978 | 5 | "01/01/09", "01/01/09", |
979 | 5 | ])) as _, |
980 | 5 | true, |
981 | 5 | ), |
982 | | ( |
983 | 5 | "string_col", |
984 | 5 | Arc::new(StringArray::from( |
985 | 5 | (0..8) |
986 | 40 | .map5 (|x| if x % 2 == 0 { "0"20 } else { "1"20 }) |
987 | 5 | .collect::<Vec<_>>(), |
988 | 5 | )) as _, |
989 | | true, |
990 | | ), |
991 | 5 | ( |
992 | 5 | "timestamp_col", |
993 | 5 | Arc::new( |
994 | 5 | TimestampMicrosecondArray::from_iter_values([ |
995 | 5 | 1235865600000000, // 2009-03-01T00:00:00.000 |
996 | 5 | 1235865660000000, // 2009-03-01T00:01:00.000 |
997 | 5 | 1238544000000000, // 2009-04-01T00:00:00.000 |
998 | 5 | 1238544060000000, // 2009-04-01T00:01:00.000 |
999 | 5 | 1233446400000000, // 2009-02-01T00:00:00.000 |
1000 | 5 | 1233446460000000, // 2009-02-01T00:01:00.000 |
1001 | 5 | 1230768000000000, // 2009-01-01T00:00:00.000 |
1002 | 5 | 1230768060000000, // 2009-01-01T00:01:00.000 |
1003 | 5 | ]) |
1004 | 5 | .with_timezone("+00:00"), |
1005 | 5 | ) as _, |
1006 | 5 | true, |
1007 | 5 | ), |
1008 | | ]) |
1009 | 5 | .unwrap(); |
1010 | 5 | assert_eq!(batch, expected, "mismatch for file {file}"0 ); |
1011 | | } |
1012 | 1 | } |
1013 | | |
1014 | | #[test] |
1015 | 1 | fn test_alltypes_schema_promotion_long_to_float_only() { |
1016 | 1 | let files = [ |
1017 | 1 | "avro/alltypes_plain.avro", |
1018 | 1 | "avro/alltypes_plain.snappy.avro", |
1019 | 1 | "avro/alltypes_plain.zstandard.avro", |
1020 | 1 | "avro/alltypes_plain.bzip2.avro", |
1021 | 1 | "avro/alltypes_plain.xz.avro", |
1022 | 1 | ]; |
1023 | 6 | for file5 in files { |
1024 | 5 | let file = arrow_test_data(file); |
1025 | 5 | let mut promotions: HashMap<&str, &str> = HashMap::new(); |
1026 | 5 | promotions.insert("bigint_col", "float"); |
1027 | 5 | let reader_schema = make_reader_schema_with_promotions(&file, &promotions); |
1028 | 5 | let batch = read_alltypes_with_reader_schema(&file, reader_schema); |
1029 | 5 | let expected = RecordBatch::try_from_iter_with_nullable([ |
1030 | 5 | ( |
1031 | 5 | "id", |
1032 | 5 | Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _, |
1033 | 5 | true, |
1034 | 5 | ), |
1035 | | ( |
1036 | 5 | "bool_col", |
1037 | 40 | Arc::new5 (BooleanArray::from_iter5 ((0..8)5 .map5 (|x| Some(x % 2 == 0)))) as _5 , |
1038 | | true, |
1039 | | ), |
1040 | | ( |
1041 | 5 | "tinyint_col", |
1042 | 40 | Arc::new5 (Int32Array::from_iter_values5 ((0..8)5 .map5 (|x| x % 2))) as _5 , |
1043 | | true, |
1044 | | ), |
1045 | | ( |
1046 | 5 | "smallint_col", |
1047 | 40 | Arc::new5 (Int32Array::from_iter_values5 ((0..8)5 .map5 (|x| x % 2))) as _5 , |
1048 | | true, |
1049 | | ), |
1050 | | ( |
1051 | 5 | "int_col", |
1052 | 40 | Arc::new5 (Int32Array::from_iter_values5 ((0..8)5 .map5 (|x| x % 2))) as _5 , |
1053 | | true, |
1054 | | ), |
1055 | | ( |
1056 | 5 | "bigint_col", |
1057 | 5 | Arc::new(Float32Array::from_iter_values( |
1058 | 40 | (0..8)5 .map5 (|x| ((x % 2) * 10) as f32), |
1059 | 5 | )) as _, |
1060 | | true, |
1061 | | ), |
1062 | | ( |
1063 | 5 | "float_col", |
1064 | 5 | Arc::new(Float32Array::from_iter_values( |
1065 | 40 | (0..8)5 .map5 (|x| (x % 2) as f32 * 1.1), |
1066 | 5 | )) as _, |
1067 | | true, |
1068 | | ), |
1069 | | ( |
1070 | 5 | "double_col", |
1071 | 5 | Arc::new(Float64Array::from_iter_values( |
1072 | 40 | (0..8)5 .map5 (|x| (x % 2) as f64 * 10.1), |
1073 | 5 | )) as _, |
1074 | | true, |
1075 | | ), |
1076 | 5 | ( |
1077 | 5 | "date_string_col", |
1078 | 5 | Arc::new(BinaryArray::from_iter_values([ |
1079 | 5 | [48, 51, 47, 48, 49, 47, 48, 57], |
1080 | 5 | [48, 51, 47, 48, 49, 47, 48, 57], |
1081 | 5 | [48, 52, 47, 48, 49, 47, 48, 57], |
1082 | 5 | [48, 52, 47, 48, 49, 47, 48, 57], |
1083 | 5 | [48, 50, 47, 48, 49, 47, 48, 57], |
1084 | 5 | [48, 50, 47, 48, 49, 47, 48, 57], |
1085 | 5 | [48, 49, 47, 48, 49, 47, 48, 57], |
1086 | 5 | [48, 49, 47, 48, 49, 47, 48, 57], |
1087 | 5 | ])) as _, |
1088 | 5 | true, |
1089 | 5 | ), |
1090 | | ( |
1091 | 5 | "string_col", |
1092 | 40 | Arc::new5 (BinaryArray::from_iter_values5 ((0..8)5 .map5 (|x| [48 + x % 2]))) as _5 , |
1093 | | true, |
1094 | | ), |
1095 | 5 | ( |
1096 | 5 | "timestamp_col", |
1097 | 5 | Arc::new( |
1098 | 5 | TimestampMicrosecondArray::from_iter_values([ |
1099 | 5 | 1235865600000000, // 2009-03-01T00:00:00.000 |
1100 | 5 | 1235865660000000, // 2009-03-01T00:01:00.000 |
1101 | 5 | 1238544000000000, // 2009-04-01T00:00:00.000 |
1102 | 5 | 1238544060000000, // 2009-04-01T00:01:00.000 |
1103 | 5 | 1233446400000000, // 2009-02-01T00:00:00.000 |
1104 | 5 | 1233446460000000, // 2009-02-01T00:01:00.000 |
1105 | 5 | 1230768000000000, // 2009-01-01T00:00:00.000 |
1106 | 5 | 1230768060000000, // 2009-01-01T00:01:00.000 |
1107 | 5 | ]) |
1108 | 5 | .with_timezone("+00:00"), |
1109 | 5 | ) as _, |
1110 | 5 | true, |
1111 | 5 | ), |
1112 | | ]) |
1113 | 5 | .unwrap(); |
1114 | 5 | assert_eq!(batch, expected, "mismatch for file {file}"0 ); |
1115 | | } |
1116 | 1 | } |
1117 | | |
1118 | | #[test] |
1119 | 1 | fn test_alltypes_schema_promotion_bytes_to_string_only() { |
1120 | 1 | let files = [ |
1121 | 1 | "avro/alltypes_plain.avro", |
1122 | 1 | "avro/alltypes_plain.snappy.avro", |
1123 | 1 | "avro/alltypes_plain.zstandard.avro", |
1124 | 1 | "avro/alltypes_plain.bzip2.avro", |
1125 | 1 | "avro/alltypes_plain.xz.avro", |
1126 | 1 | ]; |
1127 | 6 | for file5 in files { |
1128 | 5 | let file = arrow_test_data(file); |
1129 | 5 | let mut promotions: HashMap<&str, &str> = HashMap::new(); |
1130 | 5 | promotions.insert("date_string_col", "string"); |
1131 | 5 | promotions.insert("string_col", "string"); |
1132 | 5 | let reader_schema = make_reader_schema_with_promotions(&file, &promotions); |
1133 | 5 | let batch = read_alltypes_with_reader_schema(&file, reader_schema); |
1134 | 5 | let expected = RecordBatch::try_from_iter_with_nullable([ |
1135 | 5 | ( |
1136 | 5 | "id", |
1137 | 5 | Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _, |
1138 | 5 | true, |
1139 | 5 | ), |
1140 | | ( |
1141 | 5 | "bool_col", |
1142 | 40 | Arc::new5 (BooleanArray::from_iter5 ((0..8)5 .map5 (|x| Some(x % 2 == 0)))) as _5 , |
1143 | | true, |
1144 | | ), |
1145 | | ( |
1146 | 5 | "tinyint_col", |
1147 | 40 | Arc::new5 (Int32Array::from_iter_values5 ((0..8)5 .map5 (|x| x % 2))) as _5 , |
1148 | | true, |
1149 | | ), |
1150 | | ( |
1151 | 5 | "smallint_col", |
1152 | 40 | Arc::new5 (Int32Array::from_iter_values5 ((0..8)5 .map5 (|x| x % 2))) as _5 , |
1153 | | true, |
1154 | | ), |
1155 | | ( |
1156 | 5 | "int_col", |
1157 | 40 | Arc::new5 (Int32Array::from_iter_values5 ((0..8)5 .map5 (|x| x % 2))) as _5 , |
1158 | | true, |
1159 | | ), |
1160 | | ( |
1161 | 5 | "bigint_col", |
1162 | 40 | Arc::new5 (Int64Array::from_iter_values5 ((0..8)5 .map5 (|x| (x % 2) * 10))) as _5 , |
1163 | | true, |
1164 | | ), |
1165 | | ( |
1166 | 5 | "float_col", |
1167 | 5 | Arc::new(Float32Array::from_iter_values( |
1168 | 40 | (0..8)5 .map5 (|x| (x % 2) as f32 * 1.1), |
1169 | 5 | )) as _, |
1170 | | true, |
1171 | | ), |
1172 | | ( |
1173 | 5 | "double_col", |
1174 | 5 | Arc::new(Float64Array::from_iter_values( |
1175 | 40 | (0..8)5 .map5 (|x| (x % 2) as f64 * 10.1), |
1176 | 5 | )) as _, |
1177 | | true, |
1178 | | ), |
1179 | 5 | ( |
1180 | 5 | "date_string_col", |
1181 | 5 | Arc::new(StringArray::from(vec![ |
1182 | 5 | "03/01/09", "03/01/09", "04/01/09", "04/01/09", "02/01/09", "02/01/09", |
1183 | 5 | "01/01/09", "01/01/09", |
1184 | 5 | ])) as _, |
1185 | 5 | true, |
1186 | 5 | ), |
1187 | | ( |
1188 | 5 | "string_col", |
1189 | 5 | Arc::new(StringArray::from( |
1190 | 5 | (0..8) |
1191 | 40 | .map5 (|x| if x % 2 == 0 { "0"20 } else { "1"20 }) |
1192 | 5 | .collect::<Vec<_>>(), |
1193 | 5 | )) as _, |
1194 | | true, |
1195 | | ), |
1196 | 5 | ( |
1197 | 5 | "timestamp_col", |
1198 | 5 | Arc::new( |
1199 | 5 | TimestampMicrosecondArray::from_iter_values([ |
1200 | 5 | 1235865600000000, // 2009-03-01T00:00:00.000 |
1201 | 5 | 1235865660000000, // 2009-03-01T00:01:00.000 |
1202 | 5 | 1238544000000000, // 2009-04-01T00:00:00.000 |
1203 | 5 | 1238544060000000, // 2009-04-01T00:01:00.000 |
1204 | 5 | 1233446400000000, // 2009-02-01T00:00:00.000 |
1205 | 5 | 1233446460000000, // 2009-02-01T00:01:00.000 |
1206 | 5 | 1230768000000000, // 2009-01-01T00:00:00.000 |
1207 | 5 | 1230768060000000, // 2009-01-01T00:01:00.000 |
1208 | 5 | ]) |
1209 | 5 | .with_timezone("+00:00"), |
1210 | 5 | ) as _, |
1211 | 5 | true, |
1212 | 5 | ), |
1213 | | ]) |
1214 | 5 | .unwrap(); |
1215 | 5 | assert_eq!(batch, expected, "mismatch for file {file}"0 ); |
1216 | | } |
1217 | 1 | } |
1218 | | |
1219 | | #[test] |
1220 | 1 | fn test_alltypes_illegal_promotion_bool_to_double_errors() { |
1221 | 1 | let file = arrow_test_data("avro/alltypes_plain.avro"); |
1222 | 1 | let mut promotions: HashMap<&str, &str> = HashMap::new(); |
1223 | 1 | promotions.insert("bool_col", "double"); // illegal |
1224 | 1 | let reader_schema = make_reader_schema_with_promotions(&file, &promotions); |
1225 | 1 | let file_handle = File::open(&file).unwrap(); |
1226 | 1 | let result = ReaderBuilder::new() |
1227 | 1 | .with_reader_schema(reader_schema) |
1228 | 1 | .build(BufReader::new(file_handle)); |
1229 | 1 | let err = result.expect_err("expected illegal promotion to error"); |
1230 | 1 | let msg = err.to_string(); |
1231 | 1 | assert!( |
1232 | 1 | msg.contains("Illegal promotion") || msg.contains("illegal promotion")0 , |
1233 | 0 | "unexpected error: {msg}" |
1234 | | ); |
1235 | 1 | } |
1236 | | |
1237 | | #[test] |
1238 | 1 | fn test_schema_store_register_lookup() { |
1239 | 1 | let schema_int = make_record_schema(PrimitiveType::Int); |
1240 | 1 | let schema_long = make_record_schema(PrimitiveType::Long); |
1241 | 1 | let mut store = SchemaStore::new(); |
1242 | 1 | let fp_int = store.register(schema_int.clone()).unwrap(); |
1243 | 1 | let fp_long = store.register(schema_long.clone()).unwrap(); |
1244 | 1 | assert_eq!(store.lookup(&fp_int).cloned(), Some(schema_int)); |
1245 | 1 | assert_eq!(store.lookup(&fp_long).cloned(), Some(schema_long)); |
1246 | 1 | assert_eq!(store.fingerprint_algorithm(), FingerprintAlgorithm::Rabin); |
1247 | 1 | } |
1248 | | |
1249 | | #[test] |
1250 | 1 | fn test_unknown_fingerprint_is_error() { |
1251 | 1 | let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store(); |
1252 | 1 | let unknown_fp = Fingerprint::Rabin(0xDEAD_BEEF_DEAD_BEEF); |
1253 | 1 | let prefix = make_prefix(unknown_fp); |
1254 | 1 | let mut decoder = make_decoder(&store, fp_int, &schema_long); |
1255 | 1 | let err = decoder.decode(&prefix).expect_err("decode should error"); |
1256 | 1 | let msg = err.to_string(); |
1257 | 1 | assert!( |
1258 | 1 | msg.contains("Unknown fingerprint"), |
1259 | 0 | "unexpected message: {msg}" |
1260 | | ); |
1261 | 1 | } |
1262 | | |
1263 | | #[test] |
1264 | 1 | fn test_handle_prefix_incomplete_magic() { |
1265 | 1 | let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store(); |
1266 | 1 | let mut decoder = make_decoder(&store, fp_int, &schema_long); |
1267 | 1 | let buf = &SINGLE_OBJECT_MAGIC[..1]; |
1268 | 1 | let res = decoder.handle_prefix(buf).unwrap(); |
1269 | 1 | assert_eq!(res, Some(0)); |
1270 | 1 | assert!(decoder.pending_schema.is_none()); |
1271 | 1 | } |
1272 | | |
1273 | | #[test] |
1274 | 1 | fn test_handle_prefix_magic_mismatch() { |
1275 | 1 | let (store, fp_int, _fp_long, _schema_int, schema_long) = make_two_schema_store(); |
1276 | 1 | let mut decoder = make_decoder(&store, fp_int, &schema_long); |
1277 | 1 | let buf = [0xFFu8, 0x00u8, 0x01u8]; |
1278 | 1 | let res = decoder.handle_prefix(&buf).unwrap(); |
1279 | 1 | assert!(res.is_none()); |
1280 | 1 | } |
1281 | | |
1282 | | #[test] |
1283 | 1 | fn test_handle_prefix_incomplete_fingerprint() { |
1284 | 1 | let (store, fp_int, fp_long, _schema_int, schema_long) = make_two_schema_store(); |
1285 | 1 | let mut decoder = make_decoder(&store, fp_int, &schema_long); |
1286 | 1 | let long_bytes = match fp_long { |
1287 | 1 | Fingerprint::Rabin(v) => v.to_le_bytes(), |
1288 | | }; |
1289 | 1 | let mut buf = Vec::from(SINGLE_OBJECT_MAGIC); |
1290 | 1 | buf.extend_from_slice(&long_bytes[..4]); |
1291 | 1 | let res = decoder.handle_prefix(&buf).unwrap(); |
1292 | 1 | assert_eq!(res, Some(0)); |
1293 | 1 | assert!(decoder.pending_schema.is_none()); |
1294 | 1 | } |
1295 | | |
1296 | | #[test] |
1297 | 1 | fn test_handle_prefix_valid_prefix_switches_schema() { |
1298 | 1 | let (store, fp_int, fp_long, _schema_int, schema_long) = make_two_schema_store(); |
1299 | 1 | let mut decoder = make_decoder(&store, fp_int, &schema_long); |
1300 | 1 | let writer_schema_long = schema_long.schema().unwrap(); |
1301 | 1 | let root_long = AvroFieldBuilder::new(&writer_schema_long).build().unwrap(); |
1302 | 1 | let long_decoder = |
1303 | 1 | RecordDecoder::try_new_with_options(root_long.data_type(), decoder.utf8_view).unwrap(); |
1304 | 1 | let _ = decoder.cache.insert(fp_long, long_decoder); |
1305 | 1 | let mut buf = Vec::from(SINGLE_OBJECT_MAGIC); |
1306 | 1 | let Fingerprint::Rabin(v) = fp_long; |
1307 | 1 | buf.extend_from_slice(&v.to_le_bytes()); |
1308 | 1 | let consumed = decoder.handle_prefix(&buf).unwrap().unwrap(); |
1309 | 1 | assert_eq!(consumed, buf.len()); |
1310 | 1 | assert!(decoder.pending_schema.is_some()); |
1311 | 1 | assert_eq!(decoder.pending_schema.as_ref().unwrap().0, fp_long); |
1312 | 1 | } |
1313 | | |
1314 | | #[test] |
1315 | 1 | fn test_two_messages_same_schema() { |
1316 | 1 | let writer_schema = make_value_schema(PrimitiveType::Int); |
1317 | 1 | let reader_schema = writer_schema.clone(); |
1318 | 1 | let mut store = SchemaStore::new(); |
1319 | 1 | let fp = store.register(writer_schema).unwrap(); |
1320 | 1 | let msg1 = make_message(fp, 42); |
1321 | 1 | let msg2 = make_message(fp, 11); |
1322 | 1 | let input = [msg1.clone(), msg2.clone()].concat(); |
1323 | 1 | let mut decoder = ReaderBuilder::new() |
1324 | 1 | .with_batch_size(8) |
1325 | 1 | .with_reader_schema(reader_schema.clone()) |
1326 | 1 | .with_writer_schema_store(store) |
1327 | 1 | .with_active_fingerprint(fp) |
1328 | 1 | .build_decoder() |
1329 | 1 | .unwrap(); |
1330 | 1 | let _ = decoder.decode(&input).unwrap(); |
1331 | 1 | let batch = decoder.flush().unwrap().expect("batch"); |
1332 | 1 | assert_eq!(batch.num_rows(), 2); |
1333 | 1 | let col = batch |
1334 | 1 | .column(0) |
1335 | 1 | .as_any() |
1336 | 1 | .downcast_ref::<Int32Array>() |
1337 | 1 | .unwrap(); |
1338 | 1 | assert_eq!(col.value(0), 42); |
1339 | 1 | assert_eq!(col.value(1), 11); |
1340 | 1 | } |
1341 | | |
1342 | | #[test] |
1343 | 1 | fn test_two_messages_schema_switch() { |
1344 | 1 | let w_int = make_value_schema(PrimitiveType::Int); |
1345 | 1 | let w_long = make_value_schema(PrimitiveType::Long); |
1346 | 1 | let r_long = w_long.clone(); |
1347 | 1 | let mut store = SchemaStore::new(); |
1348 | 1 | let fp_int = store.register(w_int).unwrap(); |
1349 | 1 | let fp_long = store.register(w_long).unwrap(); |
1350 | 1 | let msg_int = make_message(fp_int, 1); |
1351 | 1 | let msg_long = make_message(fp_long, 123456789_i64); |
1352 | 1 | let mut decoder = ReaderBuilder::new() |
1353 | 1 | .with_batch_size(8) |
1354 | 1 | .with_writer_schema_store(store) |
1355 | 1 | .with_active_fingerprint(fp_int) |
1356 | 1 | .build_decoder() |
1357 | 1 | .unwrap(); |
1358 | 1 | let _ = decoder.decode(&msg_int).unwrap(); |
1359 | 1 | let batch1 = decoder.flush().unwrap().expect("batch1"); |
1360 | 1 | assert_eq!(batch1.num_rows(), 1); |
1361 | 1 | assert_eq!( |
1362 | 1 | batch1 |
1363 | 1 | .column(0) |
1364 | 1 | .as_any() |
1365 | 1 | .downcast_ref::<Int32Array>() |
1366 | 1 | .unwrap() |
1367 | 1 | .value(0), |
1368 | | 1 |
1369 | | ); |
1370 | 1 | let _ = decoder.decode(&msg_long).unwrap(); |
1371 | 1 | let batch2 = decoder.flush().unwrap().expect("batch2"); |
1372 | 1 | assert_eq!(batch2.num_rows(), 1); |
1373 | 1 | assert_eq!( |
1374 | 1 | batch2 |
1375 | 1 | .column(0) |
1376 | 1 | .as_any() |
1377 | 1 | .downcast_ref::<Int64Array>() |
1378 | 1 | .unwrap() |
1379 | 1 | .value(0), |
1380 | | 123456789_i64 |
1381 | | ); |
1382 | 1 | } |
1383 | | |
1384 | | #[test] |
1385 | 1 | fn test_split_message_across_chunks() { |
1386 | 1 | let writer_schema = make_value_schema(PrimitiveType::Int); |
1387 | 1 | let reader_schema = writer_schema.clone(); |
1388 | 1 | let mut store = SchemaStore::new(); |
1389 | 1 | let fp = store.register(writer_schema).unwrap(); |
1390 | 1 | let msg1 = make_message(fp, 7); |
1391 | 1 | let msg2 = make_message(fp, 8); |
1392 | 1 | let msg3 = make_message(fp, 9); |
1393 | 1 | let (pref2, body2) = msg2.split_at(10); |
1394 | 1 | let (pref3, body3) = msg3.split_at(10); |
1395 | 1 | let mut decoder = ReaderBuilder::new() |
1396 | 1 | .with_batch_size(8) |
1397 | 1 | .with_reader_schema(reader_schema) |
1398 | 1 | .with_writer_schema_store(store) |
1399 | 1 | .with_active_fingerprint(fp) |
1400 | 1 | .build_decoder() |
1401 | 1 | .unwrap(); |
1402 | 1 | let _ = decoder.decode(&msg1).unwrap(); |
1403 | 1 | let batch1 = decoder.flush().unwrap().expect("batch1"); |
1404 | 1 | assert_eq!(batch1.num_rows(), 1); |
1405 | 1 | assert_eq!( |
1406 | 1 | batch1 |
1407 | 1 | .column(0) |
1408 | 1 | .as_any() |
1409 | 1 | .downcast_ref::<Int32Array>() |
1410 | 1 | .unwrap() |
1411 | 1 | .value(0), |
1412 | | 7 |
1413 | | ); |
1414 | 1 | let _ = decoder.decode(pref2).unwrap(); |
1415 | 1 | assert!(decoder.flush().unwrap().is_none()); |
1416 | 1 | let mut chunk3 = Vec::from(body2); |
1417 | 1 | chunk3.extend_from_slice(pref3); |
1418 | 1 | let _ = decoder.decode(&chunk3).unwrap(); |
1419 | 1 | let batch2 = decoder.flush().unwrap().expect("batch2"); |
1420 | 1 | assert_eq!(batch2.num_rows(), 1); |
1421 | 1 | assert_eq!( |
1422 | 1 | batch2 |
1423 | 1 | .column(0) |
1424 | 1 | .as_any() |
1425 | 1 | .downcast_ref::<Int32Array>() |
1426 | 1 | .unwrap() |
1427 | 1 | .value(0), |
1428 | | 8 |
1429 | | ); |
1430 | 1 | let _ = decoder.decode(body3).unwrap(); |
1431 | 1 | let batch3 = decoder.flush().unwrap().expect("batch3"); |
1432 | 1 | assert_eq!(batch3.num_rows(), 1); |
1433 | 1 | assert_eq!( |
1434 | 1 | batch3 |
1435 | 1 | .column(0) |
1436 | 1 | .as_any() |
1437 | 1 | .downcast_ref::<Int32Array>() |
1438 | 1 | .unwrap() |
1439 | 1 | .value(0), |
1440 | | 9 |
1441 | | ); |
1442 | 1 | } |
1443 | | |
1444 | | #[test] |
1445 | 1 | fn test_decode_stream_with_schema() { |
1446 | | struct TestCase<'a> { |
1447 | | name: &'a str, |
1448 | | schema: &'a str, |
1449 | | expected_error: Option<&'a str>, |
1450 | | } |
1451 | 1 | let tests = vec![ |
1452 | 1 | TestCase { |
1453 | 1 | name: "success", |
1454 | 1 | schema: r#"{"type":"record","name":"test","fields":[{"name":"f2","type":"string"}]}"#, |
1455 | 1 | expected_error: None, |
1456 | 1 | }, |
1457 | 1 | TestCase { |
1458 | 1 | name: "valid schema invalid data", |
1459 | 1 | schema: r#"{"type":"record","name":"test","fields":[{"name":"f2","type":"long"}]}"#, |
1460 | 1 | expected_error: Some("did not consume all bytes"), |
1461 | 1 | }, |
1462 | | ]; |
1463 | 3 | for test2 in tests { |
1464 | 2 | let avro_schema = AvroSchema::new(test.schema.to_string()); |
1465 | 2 | let mut store = SchemaStore::new(); |
1466 | 2 | let fp = store.register(avro_schema.clone()).unwrap(); |
1467 | 2 | let prefix = make_prefix(fp); |
1468 | 2 | let record_val = "some_string"; |
1469 | 2 | let mut body = prefix; |
1470 | 2 | body.push((record_val.len() as u8) << 1); |
1471 | 2 | body.extend_from_slice(record_val.as_bytes()); |
1472 | 2 | let decoder_res = ReaderBuilder::new() |
1473 | 2 | .with_batch_size(1) |
1474 | 2 | .with_writer_schema_store(store) |
1475 | 2 | .with_active_fingerprint(fp) |
1476 | 2 | .build_decoder(); |
1477 | 2 | let decoder = match decoder_res { |
1478 | 2 | Ok(d) => d, |
1479 | 0 | Err(e) => { |
1480 | 0 | if let Some(expected) = test.expected_error { |
1481 | 0 | assert!( |
1482 | 0 | e.to_string().contains(expected), |
1483 | 0 | "Test '{}' failed at build – expected '{expected}', got '{e}'", |
1484 | | test.name |
1485 | | ); |
1486 | 0 | continue; |
1487 | | } else { |
1488 | 0 | panic!("Test '{}' failed during build: {e}", test.name); |
1489 | | } |
1490 | | } |
1491 | | }; |
1492 | 2 | let stream = Box::pin(stream::once(async { Bytes::from(body) })); |
1493 | 2 | let decoded_stream = decode_stream(decoder, stream); |
1494 | 2 | let batches_result: Result<Vec<RecordBatch>, ArrowError> = |
1495 | 2 | block_on(decoded_stream.try_collect()); |
1496 | 2 | match (batches_result, test.expected_error) { |
1497 | 1 | (Ok(batches), None) => { |
1498 | 1 | let batch = |
1499 | 1 | arrow::compute::concat_batches(&batches[0].schema(), &batches).unwrap(); |
1500 | 1 | let expected_field = Field::new("f2", DataType::Utf8, false); |
1501 | 1 | let expected_schema = Arc::new(Schema::new(vec![expected_field])); |
1502 | 1 | let expected_array = Arc::new(StringArray::from(vec![record_val])); |
1503 | 1 | let expected_batch = |
1504 | 1 | RecordBatch::try_new(expected_schema, vec![expected_array]).unwrap(); |
1505 | 1 | assert_eq!(batch, expected_batch, "Test '{}'"0 , test.name); |
1506 | | } |
1507 | 1 | (Err(e), Some(expected)) => { |
1508 | 1 | assert!( |
1509 | 1 | e.to_string().contains(expected), |
1510 | 0 | "Test '{}' – expected error containing '{expected}', got '{e}'", |
1511 | | test.name |
1512 | | ); |
1513 | | } |
1514 | 0 | (Ok(_), Some(expected)) => { |
1515 | 0 | panic!( |
1516 | 0 | "Test '{}' expected failure ('{expected}') but succeeded", |
1517 | | test.name |
1518 | | ); |
1519 | | } |
1520 | 0 | (Err(e), None) => { |
1521 | 0 | panic!("Test '{}' unexpectedly failed with '{e}'", test.name); |
1522 | | } |
1523 | | } |
1524 | | } |
1525 | 1 | } |
1526 | | |
1527 | | #[test] |
1528 | 1 | fn test_utf8view_support() { |
1529 | 1 | let schema_json = r#"{ |
1530 | 1 | "type": "record", |
1531 | 1 | "name": "test", |
1532 | 1 | "fields": [{ |
1533 | 1 | "name": "str_field", |
1534 | 1 | "type": "string" |
1535 | 1 | }] |
1536 | 1 | }"#; |
1537 | | |
1538 | 1 | let schema: crate::schema::Schema = serde_json::from_str(schema_json).unwrap(); |
1539 | 1 | let avro_field = AvroField::try_from(&schema).unwrap(); |
1540 | | |
1541 | 1 | let data_type = avro_field.data_type(); |
1542 | | |
1543 | | struct TestHelper; |
1544 | | impl TestHelper { |
1545 | 1 | fn with_utf8view(field: &Field) -> Field { |
1546 | 1 | match field.data_type() { |
1547 | | DataType::Utf8 => { |
1548 | 1 | Field::new(field.name(), DataType::Utf8View, field.is_nullable()) |
1549 | 1 | .with_metadata(field.metadata().clone()) |
1550 | | } |
1551 | 0 | _ => field.clone(), |
1552 | | } |
1553 | 1 | } |
1554 | | } |
1555 | | |
1556 | 1 | let field = TestHelper::with_utf8view(&Field::new("str_field", DataType::Utf8, false)); |
1557 | | |
1558 | 1 | assert_eq!(field.data_type(), &DataType::Utf8View); |
1559 | | |
1560 | 1 | let array = StringViewArray::from(vec!["test1", "test2"]); |
1561 | 1 | let batch = |
1562 | 1 | RecordBatch::try_from_iter(vec![("str_field", Arc::new(array) as ArrayRef)]).unwrap(); |
1563 | | |
1564 | 1 | assert!(batch.column(0).as_any().is::<StringViewArray>()); |
1565 | 1 | } |
1566 | | |
1567 | | #[test] |
1568 | 1 | fn test_alltypes_skip_writer_fields_keep_double_only() { |
1569 | 1 | let file = arrow_test_data("avro/alltypes_plain.avro"); |
1570 | 1 | let reader_schema = |
1571 | 1 | make_reader_schema_with_selected_fields_in_order(&file, &["double_col"]); |
1572 | 1 | let batch = read_alltypes_with_reader_schema(&file, reader_schema); |
1573 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([( |
1574 | | "double_col", |
1575 | 1 | Arc::new(Float64Array::from_iter_values( |
1576 | 8 | (0..8)1 .map1 (|x| (x % 2) as f64 * 10.1), |
1577 | 1 | )) as _, |
1578 | | true, |
1579 | | )]) |
1580 | 1 | .unwrap(); |
1581 | 1 | assert_eq!(batch, expected); |
1582 | 1 | } |
1583 | | |
1584 | | #[test] |
1585 | 1 | fn test_alltypes_skip_writer_fields_reorder_and_skip_many() { |
1586 | 1 | let file = arrow_test_data("avro/alltypes_plain.avro"); |
1587 | 1 | let reader_schema = |
1588 | 1 | make_reader_schema_with_selected_fields_in_order(&file, &["timestamp_col", "id"]); |
1589 | 1 | let batch = read_alltypes_with_reader_schema(&file, reader_schema); |
1590 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
1591 | 1 | ( |
1592 | 1 | "timestamp_col", |
1593 | 1 | Arc::new( |
1594 | 1 | TimestampMicrosecondArray::from_iter_values([ |
1595 | 1 | 1235865600000000, // 2009-03-01T00:00:00.000 |
1596 | 1 | 1235865660000000, // 2009-03-01T00:01:00.000 |
1597 | 1 | 1238544000000000, // 2009-04-01T00:00:00.000 |
1598 | 1 | 1238544060000000, // 2009-04-01T00:01:00.000 |
1599 | 1 | 1233446400000000, // 2009-02-01T00:00:00.000 |
1600 | 1 | 1233446460000000, // 2009-02-01T00:01:00.000 |
1601 | 1 | 1230768000000000, // 2009-01-01T00:00:00.000 |
1602 | 1 | 1230768060000000, // 2009-01-01T00:01:00.000 |
1603 | 1 | ]) |
1604 | 1 | .with_timezone("+00:00"), |
1605 | 1 | ) as _, |
1606 | 1 | true, |
1607 | 1 | ), |
1608 | 1 | ( |
1609 | 1 | "id", |
1610 | 1 | Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _, |
1611 | 1 | true, |
1612 | 1 | ), |
1613 | 1 | ]) |
1614 | 1 | .unwrap(); |
1615 | 1 | assert_eq!(batch, expected); |
1616 | 1 | } |
1617 | | |
1618 | | #[test] |
1619 | 1 | fn test_read_zero_byte_avro_file() { |
1620 | 1 | let batch = read_file("test/data/zero_byte.avro", 3, false); |
1621 | 1 | let schema = batch.schema(); |
1622 | 1 | assert_eq!(schema.fields().len(), 1); |
1623 | 1 | let field = schema.field(0); |
1624 | 1 | assert_eq!(field.name(), "data"); |
1625 | 1 | assert_eq!(field.data_type(), &DataType::Binary); |
1626 | 1 | assert!(field.is_nullable()); |
1627 | 1 | assert_eq!(batch.num_rows(), 3); |
1628 | 1 | assert_eq!(batch.num_columns(), 1); |
1629 | 1 | let binary_array = batch |
1630 | 1 | .column(0) |
1631 | 1 | .as_any() |
1632 | 1 | .downcast_ref::<BinaryArray>() |
1633 | 1 | .unwrap(); |
1634 | 1 | assert!(binary_array.is_null(0)); |
1635 | 1 | assert!(binary_array.is_valid(1)); |
1636 | 1 | assert_eq!(binary_array.value(1), b""); |
1637 | 1 | assert!(binary_array.is_valid(2)); |
1638 | 1 | assert_eq!(binary_array.value(2), b"some bytes"); |
1639 | 1 | } |
1640 | | |
1641 | | #[test] |
1642 | 1 | fn test_alltypes() { |
1643 | 1 | let files = [ |
1644 | 1 | "avro/alltypes_plain.avro", |
1645 | 1 | "avro/alltypes_plain.snappy.avro", |
1646 | 1 | "avro/alltypes_plain.zstandard.avro", |
1647 | 1 | "avro/alltypes_plain.bzip2.avro", |
1648 | 1 | "avro/alltypes_plain.xz.avro", |
1649 | 1 | ]; |
1650 | | |
1651 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
1652 | 1 | ( |
1653 | 1 | "id", |
1654 | 1 | Arc::new(Int32Array::from(vec![4, 5, 6, 7, 2, 3, 0, 1])) as _, |
1655 | 1 | true, |
1656 | 1 | ), |
1657 | | ( |
1658 | 1 | "bool_col", |
1659 | 8 | Arc::new1 (BooleanArray::from_iter1 ((0..8)1 .map1 (|x| Some(x % 2 == 0)))) as _1 , |
1660 | | true, |
1661 | | ), |
1662 | | ( |
1663 | 1 | "tinyint_col", |
1664 | 8 | Arc::new1 (Int32Array::from_iter_values1 ((0..8)1 .map1 (|x| x % 2))) as _1 , |
1665 | | true, |
1666 | | ), |
1667 | | ( |
1668 | 1 | "smallint_col", |
1669 | 8 | Arc::new1 (Int32Array::from_iter_values1 ((0..8)1 .map1 (|x| x % 2))) as _1 , |
1670 | | true, |
1671 | | ), |
1672 | | ( |
1673 | 1 | "int_col", |
1674 | 8 | Arc::new1 (Int32Array::from_iter_values1 ((0..8)1 .map1 (|x| x % 2))) as _1 , |
1675 | | true, |
1676 | | ), |
1677 | | ( |
1678 | 1 | "bigint_col", |
1679 | 8 | Arc::new1 (Int64Array::from_iter_values1 ((0..8)1 .map1 (|x| (x % 2) * 10))) as _1 , |
1680 | | true, |
1681 | | ), |
1682 | | ( |
1683 | 1 | "float_col", |
1684 | 1 | Arc::new(Float32Array::from_iter_values( |
1685 | 8 | (0..8)1 .map1 (|x| (x % 2) as f32 * 1.1), |
1686 | 1 | )) as _, |
1687 | | true, |
1688 | | ), |
1689 | | ( |
1690 | 1 | "double_col", |
1691 | 1 | Arc::new(Float64Array::from_iter_values( |
1692 | 8 | (0..8)1 .map1 (|x| (x % 2) as f64 * 10.1), |
1693 | 1 | )) as _, |
1694 | | true, |
1695 | | ), |
1696 | 1 | ( |
1697 | 1 | "date_string_col", |
1698 | 1 | Arc::new(BinaryArray::from_iter_values([ |
1699 | 1 | [48, 51, 47, 48, 49, 47, 48, 57], |
1700 | 1 | [48, 51, 47, 48, 49, 47, 48, 57], |
1701 | 1 | [48, 52, 47, 48, 49, 47, 48, 57], |
1702 | 1 | [48, 52, 47, 48, 49, 47, 48, 57], |
1703 | 1 | [48, 50, 47, 48, 49, 47, 48, 57], |
1704 | 1 | [48, 50, 47, 48, 49, 47, 48, 57], |
1705 | 1 | [48, 49, 47, 48, 49, 47, 48, 57], |
1706 | 1 | [48, 49, 47, 48, 49, 47, 48, 57], |
1707 | 1 | ])) as _, |
1708 | 1 | true, |
1709 | 1 | ), |
1710 | | ( |
1711 | 1 | "string_col", |
1712 | 8 | Arc::new1 (BinaryArray::from_iter_values1 ((0..8)1 .map1 (|x| [48 + x % 2]))) as _1 , |
1713 | | true, |
1714 | | ), |
1715 | 1 | ( |
1716 | 1 | "timestamp_col", |
1717 | 1 | Arc::new( |
1718 | 1 | TimestampMicrosecondArray::from_iter_values([ |
1719 | 1 | 1235865600000000, // 2009-03-01T00:00:00.000 |
1720 | 1 | 1235865660000000, // 2009-03-01T00:01:00.000 |
1721 | 1 | 1238544000000000, // 2009-04-01T00:00:00.000 |
1722 | 1 | 1238544060000000, // 2009-04-01T00:01:00.000 |
1723 | 1 | 1233446400000000, // 2009-02-01T00:00:00.000 |
1724 | 1 | 1233446460000000, // 2009-02-01T00:01:00.000 |
1725 | 1 | 1230768000000000, // 2009-01-01T00:00:00.000 |
1726 | 1 | 1230768060000000, // 2009-01-01T00:01:00.000 |
1727 | 1 | ]) |
1728 | 1 | .with_timezone("+00:00"), |
1729 | 1 | ) as _, |
1730 | 1 | true, |
1731 | 1 | ), |
1732 | | ]) |
1733 | 1 | .unwrap(); |
1734 | | |
1735 | 6 | for file5 in files { |
1736 | 5 | let file = arrow_test_data(file); |
1737 | | |
1738 | 5 | assert_eq!(read_file(&file, 8, false), expected); |
1739 | 5 | assert_eq!(read_file(&file, 3, false), expected); |
1740 | | } |
1741 | 1 | } |
1742 | | |
1743 | | #[test] |
1744 | 1 | fn test_alltypes_dictionary() { |
1745 | 1 | let file = "avro/alltypes_dictionary.avro"; |
1746 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
1747 | 1 | ("id", Arc::new(Int32Array::from(vec![0, 1])) as _, true), |
1748 | 1 | ( |
1749 | 1 | "bool_col", |
1750 | 1 | Arc::new(BooleanArray::from(vec![Some(true), Some(false)])) as _, |
1751 | 1 | true, |
1752 | 1 | ), |
1753 | 1 | ( |
1754 | 1 | "tinyint_col", |
1755 | 1 | Arc::new(Int32Array::from(vec![0, 1])) as _, |
1756 | 1 | true, |
1757 | 1 | ), |
1758 | 1 | ( |
1759 | 1 | "smallint_col", |
1760 | 1 | Arc::new(Int32Array::from(vec![0, 1])) as _, |
1761 | 1 | true, |
1762 | 1 | ), |
1763 | 1 | ("int_col", Arc::new(Int32Array::from(vec![0, 1])) as _, true), |
1764 | 1 | ( |
1765 | 1 | "bigint_col", |
1766 | 1 | Arc::new(Int64Array::from(vec![0, 10])) as _, |
1767 | 1 | true, |
1768 | 1 | ), |
1769 | 1 | ( |
1770 | 1 | "float_col", |
1771 | 1 | Arc::new(Float32Array::from(vec![0.0, 1.1])) as _, |
1772 | 1 | true, |
1773 | 1 | ), |
1774 | 1 | ( |
1775 | 1 | "double_col", |
1776 | 1 | Arc::new(Float64Array::from(vec![0.0, 10.1])) as _, |
1777 | 1 | true, |
1778 | 1 | ), |
1779 | 1 | ( |
1780 | 1 | "date_string_col", |
1781 | 1 | Arc::new(BinaryArray::from_iter_values([b"01/01/09", b"01/01/09"])) as _, |
1782 | 1 | true, |
1783 | 1 | ), |
1784 | 1 | ( |
1785 | 1 | "string_col", |
1786 | 1 | Arc::new(BinaryArray::from_iter_values([b"0", b"1"])) as _, |
1787 | 1 | true, |
1788 | 1 | ), |
1789 | 1 | ( |
1790 | 1 | "timestamp_col", |
1791 | 1 | Arc::new( |
1792 | 1 | TimestampMicrosecondArray::from_iter_values([ |
1793 | 1 | 1230768000000000, // 2009-01-01T00:00:00.000 |
1794 | 1 | 1230768060000000, // 2009-01-01T00:01:00.000 |
1795 | 1 | ]) |
1796 | 1 | .with_timezone("+00:00"), |
1797 | 1 | ) as _, |
1798 | 1 | true, |
1799 | 1 | ), |
1800 | 1 | ]) |
1801 | 1 | .unwrap(); |
1802 | 1 | let file_path = arrow_test_data(file); |
1803 | 1 | let batch_large = read_file(&file_path, 8, false); |
1804 | 1 | assert_eq!( |
1805 | | batch_large, expected, |
1806 | 0 | "Decoded RecordBatch does not match for file {file}" |
1807 | | ); |
1808 | 1 | let batch_small = read_file(&file_path, 3, false); |
1809 | 1 | assert_eq!( |
1810 | | batch_small, expected, |
1811 | 0 | "Decoded RecordBatch (batch size 3) does not match for file {file}" |
1812 | | ); |
1813 | 1 | } |
1814 | | |
1815 | | #[test] |
1816 | 1 | fn test_alltypes_nulls_plain() { |
1817 | 1 | let file = "avro/alltypes_nulls_plain.avro"; |
1818 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
1819 | 1 | ( |
1820 | 1 | "string_col", |
1821 | 1 | Arc::new(StringArray::from(vec![None::<&str>])) as _, |
1822 | 1 | true, |
1823 | 1 | ), |
1824 | 1 | ("int_col", Arc::new(Int32Array::from(vec![None])) as _, true), |
1825 | 1 | ( |
1826 | 1 | "bool_col", |
1827 | 1 | Arc::new(BooleanArray::from(vec![None])) as _, |
1828 | 1 | true, |
1829 | 1 | ), |
1830 | 1 | ( |
1831 | 1 | "bigint_col", |
1832 | 1 | Arc::new(Int64Array::from(vec![None])) as _, |
1833 | 1 | true, |
1834 | 1 | ), |
1835 | 1 | ( |
1836 | 1 | "float_col", |
1837 | 1 | Arc::new(Float32Array::from(vec![None])) as _, |
1838 | 1 | true, |
1839 | 1 | ), |
1840 | 1 | ( |
1841 | 1 | "double_col", |
1842 | 1 | Arc::new(Float64Array::from(vec![None])) as _, |
1843 | 1 | true, |
1844 | 1 | ), |
1845 | 1 | ( |
1846 | 1 | "bytes_col", |
1847 | 1 | Arc::new(BinaryArray::from(vec![None::<&[u8]>])) as _, |
1848 | 1 | true, |
1849 | 1 | ), |
1850 | 1 | ]) |
1851 | 1 | .unwrap(); |
1852 | 1 | let file_path = arrow_test_data(file); |
1853 | 1 | let batch_large = read_file(&file_path, 8, false); |
1854 | 1 | assert_eq!( |
1855 | | batch_large, expected, |
1856 | 0 | "Decoded RecordBatch does not match for file {file}" |
1857 | | ); |
1858 | 1 | let batch_small = read_file(&file_path, 3, false); |
1859 | 1 | assert_eq!( |
1860 | | batch_small, expected, |
1861 | 0 | "Decoded RecordBatch (batch size 3) does not match for file {file}" |
1862 | | ); |
1863 | 1 | } |
1864 | | |
1865 | | #[test] |
1866 | 1 | fn test_binary() { |
1867 | 1 | let file = arrow_test_data("avro/binary.avro"); |
1868 | 1 | let batch = read_file(&file, 8, false); |
1869 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([( |
1870 | 1 | "foo", |
1871 | 1 | Arc::new(BinaryArray::from_iter_values(vec![ |
1872 | 1 | b"\x00".as_ref(), |
1873 | 1 | b"\x01".as_ref(), |
1874 | 1 | b"\x02".as_ref(), |
1875 | 1 | b"\x03".as_ref(), |
1876 | 1 | b"\x04".as_ref(), |
1877 | 1 | b"\x05".as_ref(), |
1878 | 1 | b"\x06".as_ref(), |
1879 | 1 | b"\x07".as_ref(), |
1880 | 1 | b"\x08".as_ref(), |
1881 | 1 | b"\t".as_ref(), |
1882 | 1 | b"\n".as_ref(), |
1883 | 1 | b"\x0b".as_ref(), |
1884 | 1 | ])) as Arc<dyn Array>, |
1885 | 1 | true, |
1886 | 1 | )]) |
1887 | 1 | .unwrap(); |
1888 | 1 | assert_eq!(batch, expected); |
1889 | 1 | } |
1890 | | |
1891 | | #[test] |
1892 | 1 | fn test_decimal() { |
1893 | 1 | let files = [ |
1894 | 1 | ("avro/fixed_length_decimal.avro", 25, 2), |
1895 | 1 | ("avro/fixed_length_decimal_legacy.avro", 13, 2), |
1896 | 1 | ("avro/int32_decimal.avro", 4, 2), |
1897 | 1 | ("avro/int64_decimal.avro", 10, 2), |
1898 | 1 | ]; |
1899 | 24 | let decimal_values1 : Vec<i128>1 = (1..=241 ).map1 (|n| n as i128 * 100).collect1 (); |
1900 | 5 | for (file4 , precision4 , scale4 ) in files { |
1901 | 4 | let file_path = arrow_test_data(file); |
1902 | 4 | let actual_batch = read_file(&file_path, 8, false); |
1903 | 4 | let expected_array = Decimal128Array::from_iter_values(decimal_values.clone()) |
1904 | 4 | .with_precision_and_scale(precision, scale) |
1905 | 4 | .unwrap(); |
1906 | 4 | let mut meta = HashMap::new(); |
1907 | 4 | meta.insert("precision".to_string(), precision.to_string()); |
1908 | 4 | meta.insert("scale".to_string(), scale.to_string()); |
1909 | 4 | let field_with_meta = Field::new("value", DataType::Decimal128(precision, scale), true) |
1910 | 4 | .with_metadata(meta); |
1911 | 4 | let expected_schema = Arc::new(Schema::new(vec![field_with_meta])); |
1912 | 4 | let expected_batch = |
1913 | 4 | RecordBatch::try_new(expected_schema.clone(), vec![Arc::new(expected_array)]) |
1914 | 4 | .expect("Failed to build expected RecordBatch"); |
1915 | 4 | assert_eq!( |
1916 | | actual_batch, expected_batch, |
1917 | 0 | "Decoded RecordBatch does not match the expected Decimal128 data for file {file}" |
1918 | | ); |
1919 | 4 | let actual_batch_small = read_file(&file_path, 3, false); |
1920 | 4 | assert_eq!( |
1921 | | actual_batch_small, |
1922 | | expected_batch, |
1923 | 0 | "Decoded RecordBatch does not match the expected Decimal128 data for file {file} with batch size 3" |
1924 | | ); |
1925 | | } |
1926 | 1 | } |
1927 | | |
1928 | | #[test] |
1929 | 1 | fn test_dict_pages_offset_zero() { |
1930 | 1 | let file = arrow_test_data("avro/dict-page-offset-zero.avro"); |
1931 | 1 | let batch = read_file(&file, 32, false); |
1932 | 1 | let num_rows = batch.num_rows(); |
1933 | 1 | let expected_field = Int32Array::from(vec![Some(1552); num_rows]); |
1934 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([( |
1935 | 1 | "l_partkey", |
1936 | 1 | Arc::new(expected_field) as Arc<dyn Array>, |
1937 | 1 | true, |
1938 | 1 | )]) |
1939 | 1 | .unwrap(); |
1940 | 1 | assert_eq!(batch, expected); |
1941 | 1 | } |
1942 | | |
1943 | | #[test] |
1944 | 1 | fn test_list_columns() { |
1945 | 1 | let file = arrow_test_data("avro/list_columns.avro"); |
1946 | 1 | let mut int64_list_builder = ListBuilder::new(Int64Builder::new()); |
1947 | 1 | { |
1948 | 1 | { |
1949 | 1 | let values = int64_list_builder.values(); |
1950 | 1 | values.append_value(1); |
1951 | 1 | values.append_value(2); |
1952 | 1 | values.append_value(3); |
1953 | 1 | } |
1954 | 1 | int64_list_builder.append(true); |
1955 | 1 | } |
1956 | 1 | { |
1957 | 1 | { |
1958 | 1 | let values = int64_list_builder.values(); |
1959 | 1 | values.append_null(); |
1960 | 1 | values.append_value(1); |
1961 | 1 | } |
1962 | 1 | int64_list_builder.append(true); |
1963 | 1 | } |
1964 | 1 | { |
1965 | 1 | { |
1966 | 1 | let values = int64_list_builder.values(); |
1967 | 1 | values.append_value(4); |
1968 | 1 | } |
1969 | 1 | int64_list_builder.append(true); |
1970 | 1 | } |
1971 | 1 | let int64_list = int64_list_builder.finish(); |
1972 | 1 | let mut utf8_list_builder = ListBuilder::new(StringBuilder::new()); |
1973 | 1 | { |
1974 | 1 | { |
1975 | 1 | let values = utf8_list_builder.values(); |
1976 | 1 | values.append_value("abc"); |
1977 | 1 | values.append_value("efg"); |
1978 | 1 | values.append_value("hij"); |
1979 | 1 | } |
1980 | 1 | utf8_list_builder.append(true); |
1981 | 1 | } |
1982 | 1 | { |
1983 | 1 | utf8_list_builder.append(false); |
1984 | 1 | } |
1985 | 1 | { |
1986 | 1 | { |
1987 | 1 | let values = utf8_list_builder.values(); |
1988 | 1 | values.append_value("efg"); |
1989 | 1 | values.append_null(); |
1990 | 1 | values.append_value("hij"); |
1991 | 1 | values.append_value("xyz"); |
1992 | 1 | } |
1993 | 1 | utf8_list_builder.append(true); |
1994 | 1 | } |
1995 | 1 | let utf8_list = utf8_list_builder.finish(); |
1996 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
1997 | 1 | ("int64_list", Arc::new(int64_list) as Arc<dyn Array>, true), |
1998 | 1 | ("utf8_list", Arc::new(utf8_list) as Arc<dyn Array>, true), |
1999 | 1 | ]) |
2000 | 1 | .unwrap(); |
2001 | 1 | let batch = read_file(&file, 8, false); |
2002 | 1 | assert_eq!(batch, expected); |
2003 | 1 | } |
2004 | | |
2005 | | #[test] |
2006 | 1 | fn test_nested_lists() { |
2007 | | use arrow_data::ArrayDataBuilder; |
2008 | 1 | let file = arrow_test_data("avro/nested_lists.snappy.avro"); |
2009 | 1 | let inner_values = StringArray::from(vec![ |
2010 | 1 | Some("a"), |
2011 | 1 | Some("b"), |
2012 | 1 | Some("c"), |
2013 | 1 | Some("d"), |
2014 | 1 | Some("a"), |
2015 | 1 | Some("b"), |
2016 | 1 | Some("c"), |
2017 | 1 | Some("d"), |
2018 | 1 | Some("e"), |
2019 | 1 | Some("a"), |
2020 | 1 | Some("b"), |
2021 | 1 | Some("c"), |
2022 | 1 | Some("d"), |
2023 | 1 | Some("e"), |
2024 | 1 | Some("f"), |
2025 | | ]); |
2026 | 1 | let inner_offsets = Buffer::from_slice_ref([0, 2, 3, 3, 4, 6, 8, 8, 9, 11, 13, 14, 14, 15]); |
2027 | 1 | let inner_validity = [ |
2028 | 1 | true, true, false, true, true, true, false, true, true, true, true, false, true, |
2029 | 1 | ]; |
2030 | 1 | let inner_null_buffer = Buffer::from_iter(inner_validity.iter().copied()); |
2031 | 1 | let inner_field = Field::new("item", DataType::Utf8, true); |
2032 | 1 | let inner_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(inner_field))) |
2033 | 1 | .len(13) |
2034 | 1 | .add_buffer(inner_offsets) |
2035 | 1 | .add_child_data(inner_values.to_data()) |
2036 | 1 | .null_bit_buffer(Some(inner_null_buffer)) |
2037 | 1 | .build() |
2038 | 1 | .unwrap(); |
2039 | 1 | let inner_list_array = ListArray::from(inner_list_data); |
2040 | 1 | let middle_offsets = Buffer::from_slice_ref([0, 2, 4, 6, 8, 11, 13]); |
2041 | 1 | let middle_validity = [true; 6]; |
2042 | 1 | let middle_null_buffer = Buffer::from_iter(middle_validity.iter().copied()); |
2043 | 1 | let middle_field = Field::new("item", inner_list_array.data_type().clone(), true); |
2044 | 1 | let middle_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(middle_field))) |
2045 | 1 | .len(6) |
2046 | 1 | .add_buffer(middle_offsets) |
2047 | 1 | .add_child_data(inner_list_array.to_data()) |
2048 | 1 | .null_bit_buffer(Some(middle_null_buffer)) |
2049 | 1 | .build() |
2050 | 1 | .unwrap(); |
2051 | 1 | let middle_list_array = ListArray::from(middle_list_data); |
2052 | 1 | let outer_offsets = Buffer::from_slice_ref([0, 2, 4, 6]); |
2053 | 1 | let outer_null_buffer = Buffer::from_slice_ref([0b111]); // all 3 rows valid |
2054 | 1 | let outer_field = Field::new("item", middle_list_array.data_type().clone(), true); |
2055 | 1 | let outer_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(outer_field))) |
2056 | 1 | .len(3) |
2057 | 1 | .add_buffer(outer_offsets) |
2058 | 1 | .add_child_data(middle_list_array.to_data()) |
2059 | 1 | .null_bit_buffer(Some(outer_null_buffer)) |
2060 | 1 | .build() |
2061 | 1 | .unwrap(); |
2062 | 1 | let a_expected = ListArray::from(outer_list_data); |
2063 | 1 | let b_expected = Int32Array::from(vec![1, 1, 1]); |
2064 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
2065 | 1 | ("a", Arc::new(a_expected) as Arc<dyn Array>, true), |
2066 | 1 | ("b", Arc::new(b_expected) as Arc<dyn Array>, true), |
2067 | 1 | ]) |
2068 | 1 | .unwrap(); |
2069 | 1 | let left = read_file(&file, 8, false); |
2070 | 1 | assert_eq!(left, expected, "Mismatch for batch size=8"0 ); |
2071 | 1 | let left_small = read_file(&file, 3, false); |
2072 | 1 | assert_eq!(left_small, expected, "Mismatch for batch size=3"0 ); |
2073 | 1 | } |
2074 | | |
2075 | | #[test] |
2076 | 1 | fn test_simple() { |
2077 | 1 | let tests = [ |
2078 | 1 | ("avro/simple_enum.avro", 4, build_expected_enum(), 2), |
2079 | 1 | ("avro/simple_fixed.avro", 2, build_expected_fixed(), 1), |
2080 | 1 | ]; |
2081 | | |
2082 | 1 | fn build_expected_enum() -> RecordBatch { |
2083 | | // Build the DictionaryArrays for f1, f2, f3 |
2084 | 1 | let keys_f1 = Int32Array::from(vec![0, 1, 2, 3]); |
2085 | 1 | let vals_f1 = StringArray::from(vec!["a", "b", "c", "d"]); |
2086 | 1 | let f1_dict = |
2087 | 1 | DictionaryArray::<Int32Type>::try_new(keys_f1, Arc::new(vals_f1)).unwrap(); |
2088 | 1 | let keys_f2 = Int32Array::from(vec![2, 3, 0, 1]); |
2089 | 1 | let vals_f2 = StringArray::from(vec!["e", "f", "g", "h"]); |
2090 | 1 | let f2_dict = |
2091 | 1 | DictionaryArray::<Int32Type>::try_new(keys_f2, Arc::new(vals_f2)).unwrap(); |
2092 | 1 | let keys_f3 = Int32Array::from(vec![Some(1), Some(2), None, Some(0)]); |
2093 | 1 | let vals_f3 = StringArray::from(vec!["i", "j", "k"]); |
2094 | 1 | let f3_dict = |
2095 | 1 | DictionaryArray::<Int32Type>::try_new(keys_f3, Arc::new(vals_f3)).unwrap(); |
2096 | 1 | let dict_type = |
2097 | 1 | DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); |
2098 | 1 | let mut md_f1 = HashMap::new(); |
2099 | 1 | md_f1.insert( |
2100 | 1 | AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(), |
2101 | 1 | r#"["a","b","c","d"]"#.to_string(), |
2102 | | ); |
2103 | 1 | let f1_field = Field::new("f1", dict_type.clone(), false).with_metadata(md_f1); |
2104 | 1 | let mut md_f2 = HashMap::new(); |
2105 | 1 | md_f2.insert( |
2106 | 1 | AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(), |
2107 | 1 | r#"["e","f","g","h"]"#.to_string(), |
2108 | | ); |
2109 | 1 | let f2_field = Field::new("f2", dict_type.clone(), false).with_metadata(md_f2); |
2110 | 1 | let mut md_f3 = HashMap::new(); |
2111 | 1 | md_f3.insert( |
2112 | 1 | AVRO_ENUM_SYMBOLS_METADATA_KEY.to_string(), |
2113 | 1 | r#"["i","j","k"]"#.to_string(), |
2114 | | ); |
2115 | 1 | let f3_field = Field::new("f3", dict_type.clone(), true).with_metadata(md_f3); |
2116 | 1 | let expected_schema = Arc::new(Schema::new(vec![f1_field, f2_field, f3_field])); |
2117 | 1 | RecordBatch::try_new( |
2118 | 1 | expected_schema, |
2119 | 1 | vec![ |
2120 | 1 | Arc::new(f1_dict) as Arc<dyn Array>, |
2121 | 1 | Arc::new(f2_dict) as Arc<dyn Array>, |
2122 | 1 | Arc::new(f3_dict) as Arc<dyn Array>, |
2123 | | ], |
2124 | | ) |
2125 | 1 | .unwrap() |
2126 | 1 | } |
2127 | | |
2128 | 1 | fn build_expected_fixed() -> RecordBatch { |
2129 | 1 | let f1 = |
2130 | 1 | FixedSizeBinaryArray::try_from_iter(vec![b"abcde", b"12345"].into_iter()).unwrap(); |
2131 | 1 | let f2 = |
2132 | 1 | FixedSizeBinaryArray::try_from_iter(vec![b"fghijklmno", b"1234567890"].into_iter()) |
2133 | 1 | .unwrap(); |
2134 | 1 | let f3 = FixedSizeBinaryArray::try_from_sparse_iter_with_size( |
2135 | 1 | vec![Some(b"ABCDEF" as &[u8]), None].into_iter(), |
2136 | | 6, |
2137 | | ) |
2138 | 1 | .unwrap(); |
2139 | 1 | let expected_schema = Arc::new(Schema::new(vec![ |
2140 | 1 | Field::new("f1", DataType::FixedSizeBinary(5), false), |
2141 | 1 | Field::new("f2", DataType::FixedSizeBinary(10), false), |
2142 | 1 | Field::new("f3", DataType::FixedSizeBinary(6), true), |
2143 | | ])); |
2144 | 1 | RecordBatch::try_new( |
2145 | 1 | expected_schema, |
2146 | 1 | vec![ |
2147 | 1 | Arc::new(f1) as Arc<dyn Array>, |
2148 | 1 | Arc::new(f2) as Arc<dyn Array>, |
2149 | 1 | Arc::new(f3) as Arc<dyn Array>, |
2150 | | ], |
2151 | | ) |
2152 | 1 | .unwrap() |
2153 | 1 | } |
2154 | 3 | for (file_name2 , batch_size2 , expected2 , alt_batch_size2 ) in tests { |
2155 | 2 | let file = arrow_test_data(file_name); |
2156 | 2 | let actual = read_file(&file, batch_size, false); |
2157 | 2 | assert_eq!(actual, expected); |
2158 | 2 | let actual2 = read_file(&file, alt_batch_size, false); |
2159 | 2 | assert_eq!(actual2, expected); |
2160 | | } |
2161 | 1 | } |
2162 | | |
2163 | | #[test] |
2164 | 1 | fn test_single_nan() { |
2165 | 1 | let file = arrow_test_data("avro/single_nan.avro"); |
2166 | 1 | let actual = read_file(&file, 1, false); |
2167 | | use arrow_array::Float64Array; |
2168 | 1 | let schema = Arc::new(Schema::new(vec![Field::new( |
2169 | | "mycol", |
2170 | 1 | DataType::Float64, |
2171 | | true, |
2172 | | )])); |
2173 | 1 | let col = Float64Array::from(vec![None]); |
2174 | 1 | let expected = RecordBatch::try_new(schema, vec![Arc::new(col)]).unwrap(); |
2175 | 1 | assert_eq!(actual, expected); |
2176 | 1 | let actual2 = read_file(&file, 2, false); |
2177 | 1 | assert_eq!(actual2, expected); |
2178 | 1 | } |
2179 | | |
2180 | | #[test] |
2181 | 1 | fn test_duration_uuid() { |
2182 | 1 | let batch = read_file("test/data/duration_uuid.avro", 4, false); |
2183 | 1 | let schema = batch.schema(); |
2184 | 1 | let fields = schema.fields(); |
2185 | 1 | assert_eq!(fields.len(), 2); |
2186 | 1 | assert_eq!(fields[0].name(), "duration_field"); |
2187 | 1 | assert_eq!( |
2188 | 1 | fields[0].data_type(), |
2189 | | &DataType::Interval(IntervalUnit::MonthDayNano) |
2190 | | ); |
2191 | 1 | assert_eq!(fields[1].name(), "uuid_field"); |
2192 | 1 | assert_eq!(fields[1].data_type(), &DataType::FixedSizeBinary(16)); |
2193 | 1 | assert_eq!(batch.num_rows(), 4); |
2194 | 1 | assert_eq!(batch.num_columns(), 2); |
2195 | 1 | let duration_array = batch |
2196 | 1 | .column(0) |
2197 | 1 | .as_any() |
2198 | 1 | .downcast_ref::<IntervalMonthDayNanoArray>() |
2199 | 1 | .unwrap(); |
2200 | 1 | let expected_duration_array: IntervalMonthDayNanoArray = [ |
2201 | 1 | Some(IntervalMonthDayNanoType::make_value(1, 15, 500_000_000)), |
2202 | 1 | Some(IntervalMonthDayNanoType::make_value(0, 5, 2_500_000_000)), |
2203 | 1 | Some(IntervalMonthDayNanoType::make_value(2, 0, 0)), |
2204 | 1 | Some(IntervalMonthDayNanoType::make_value(12, 31, 999_000_000)), |
2205 | 1 | ] |
2206 | 1 | .iter() |
2207 | 1 | .copied() |
2208 | 1 | .collect(); |
2209 | 1 | assert_eq!(&expected_duration_array, duration_array); |
2210 | 1 | let uuid_array = batch |
2211 | 1 | .column(1) |
2212 | 1 | .as_any() |
2213 | 1 | .downcast_ref::<FixedSizeBinaryArray>() |
2214 | 1 | .unwrap(); |
2215 | 1 | let expected_uuid_array = FixedSizeBinaryArray::try_from_sparse_iter_with_size( |
2216 | 1 | [ |
2217 | 1 | Some([ |
2218 | 1 | 0xfe, 0x7b, 0xc3, 0x0b, 0x4c, 0xe8, 0x4c, 0x5e, 0xb6, 0x7c, 0x22, 0x34, 0xa2, |
2219 | 1 | 0xd3, 0x8e, 0x66, |
2220 | 1 | ]), |
2221 | 1 | Some([ |
2222 | 1 | 0xb3, 0x3f, 0x2a, 0xd7, 0x97, 0xb4, 0x4d, 0xe1, 0x8b, 0xfe, 0x94, 0x94, 0x1d, |
2223 | 1 | 0x60, 0x15, 0x6e, |
2224 | 1 | ]), |
2225 | 1 | Some([ |
2226 | 1 | 0x5f, 0x74, 0x92, 0x64, 0x07, 0x4b, 0x40, 0x05, 0x84, 0xbf, 0x11, 0x5e, 0xa8, |
2227 | 1 | 0x4e, 0xd2, 0x0a, |
2228 | 1 | ]), |
2229 | 1 | Some([ |
2230 | 1 | 0x08, 0x26, 0xcc, 0x06, 0xd2, 0xe3, 0x45, 0x99, 0xb4, 0xad, 0xaf, 0x5f, 0xa6, |
2231 | 1 | 0x90, 0x5c, 0xdb, |
2232 | 1 | ]), |
2233 | 1 | ] |
2234 | 1 | .into_iter(), |
2235 | | 16, |
2236 | | ) |
2237 | 1 | .unwrap(); |
2238 | 1 | assert_eq!(&expected_uuid_array, uuid_array); |
2239 | 1 | } |
2240 | | |
2241 | | #[test] |
2242 | 1 | fn test_datapage_v2() { |
2243 | 1 | let file = arrow_test_data("avro/datapage_v2.snappy.avro"); |
2244 | 1 | let batch = read_file(&file, 8, false); |
2245 | 1 | let a = StringArray::from(vec![ |
2246 | 1 | Some("abc"), |
2247 | 1 | Some("abc"), |
2248 | 1 | Some("abc"), |
2249 | 1 | None, |
2250 | 1 | Some("abc"), |
2251 | | ]); |
2252 | 1 | let b = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4), Some(5)]); |
2253 | 1 | let c = Float64Array::from(vec![Some(2.0), Some(3.0), Some(4.0), Some(5.0), Some(2.0)]); |
2254 | 1 | let d = BooleanArray::from(vec![ |
2255 | 1 | Some(true), |
2256 | 1 | Some(true), |
2257 | 1 | Some(true), |
2258 | 1 | Some(false), |
2259 | 1 | Some(true), |
2260 | | ]); |
2261 | 1 | let e_values = Int32Array::from(vec![ |
2262 | 1 | Some(1), |
2263 | 1 | Some(2), |
2264 | 1 | Some(3), |
2265 | 1 | Some(1), |
2266 | 1 | Some(2), |
2267 | 1 | Some(3), |
2268 | 1 | Some(1), |
2269 | 1 | Some(2), |
2270 | | ]); |
2271 | 1 | let e_offsets = OffsetBuffer::new(ScalarBuffer::from(vec![0i32, 3, 3, 3, 6, 8])); |
2272 | 1 | let e_validity = Some(NullBuffer::from(vec![true, false, false, true, true])); |
2273 | 1 | let field_e = Arc::new(Field::new("item", DataType::Int32, true)); |
2274 | 1 | let e = ListArray::new(field_e, e_offsets, Arc::new(e_values), e_validity); |
2275 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
2276 | 1 | ("a", Arc::new(a) as Arc<dyn Array>, true), |
2277 | 1 | ("b", Arc::new(b) as Arc<dyn Array>, true), |
2278 | 1 | ("c", Arc::new(c) as Arc<dyn Array>, true), |
2279 | 1 | ("d", Arc::new(d) as Arc<dyn Array>, true), |
2280 | 1 | ("e", Arc::new(e) as Arc<dyn Array>, true), |
2281 | 1 | ]) |
2282 | 1 | .unwrap(); |
2283 | 1 | assert_eq!(batch, expected); |
2284 | 1 | } |
2285 | | |
2286 | | #[test] |
2287 | 1 | fn test_nested_records() { |
2288 | 1 | let f1_f1_1 = StringArray::from(vec!["aaa", "bbb"]); |
2289 | 1 | let f1_f1_2 = Int32Array::from(vec![10, 20]); |
2290 | 1 | let rounded_pi = (std::f64::consts::PI * 100.0).round() / 100.0; |
2291 | 1 | let f1_f1_3_1 = Float64Array::from(vec![rounded_pi, rounded_pi]); |
2292 | 1 | let f1_f1_3 = StructArray::from(vec![( |
2293 | 1 | Arc::new(Field::new("f1_3_1", DataType::Float64, false)), |
2294 | 1 | Arc::new(f1_f1_3_1) as Arc<dyn Array>, |
2295 | 1 | )]); |
2296 | 1 | let f1_expected = StructArray::from(vec![ |
2297 | 1 | ( |
2298 | 1 | Arc::new(Field::new("f1_1", DataType::Utf8, false)), |
2299 | 1 | Arc::new(f1_f1_1) as Arc<dyn Array>, |
2300 | 1 | ), |
2301 | 1 | ( |
2302 | 1 | Arc::new(Field::new("f1_2", DataType::Int32, false)), |
2303 | 1 | Arc::new(f1_f1_2) as Arc<dyn Array>, |
2304 | 1 | ), |
2305 | 1 | ( |
2306 | 1 | Arc::new(Field::new( |
2307 | 1 | "f1_3", |
2308 | 1 | DataType::Struct(Fields::from(vec![Field::new( |
2309 | 1 | "f1_3_1", |
2310 | 1 | DataType::Float64, |
2311 | 1 | false, |
2312 | 1 | )])), |
2313 | 1 | false, |
2314 | 1 | )), |
2315 | 1 | Arc::new(f1_f1_3) as Arc<dyn Array>, |
2316 | 1 | ), |
2317 | | ]); |
2318 | | |
2319 | 1 | let f2_fields = vec![ |
2320 | 1 | Field::new("f2_1", DataType::Boolean, false), |
2321 | 1 | Field::new("f2_2", DataType::Float32, false), |
2322 | | ]; |
2323 | 1 | let f2_struct_builder = StructBuilder::new( |
2324 | 1 | f2_fields |
2325 | 1 | .iter() |
2326 | 2 | .map1 (|f| Arc::new(f.clone())) |
2327 | 1 | .collect::<Vec<Arc<Field>>>(), |
2328 | 1 | vec![ |
2329 | 1 | Box::new(BooleanBuilder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>, |
2330 | 1 | Box::new(Float32Builder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>, |
2331 | | ], |
2332 | | ); |
2333 | 1 | let mut f2_list_builder = ListBuilder::new(f2_struct_builder); |
2334 | 1 | { |
2335 | 1 | let struct_builder = f2_list_builder.values(); |
2336 | 1 | struct_builder.append(true); |
2337 | 1 | { |
2338 | 1 | let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap(); |
2339 | 1 | b.append_value(true); |
2340 | 1 | } |
2341 | 1 | { |
2342 | 1 | let b = struct_builder.field_builder::<Float32Builder>(1).unwrap(); |
2343 | 1 | b.append_value(1.2_f32); |
2344 | 1 | } |
2345 | 1 | struct_builder.append(true); |
2346 | 1 | { |
2347 | 1 | let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap(); |
2348 | 1 | b.append_value(true); |
2349 | 1 | } |
2350 | 1 | { |
2351 | 1 | let b = struct_builder.field_builder::<Float32Builder>(1).unwrap(); |
2352 | 1 | b.append_value(2.2_f32); |
2353 | 1 | } |
2354 | 1 | f2_list_builder.append(true); |
2355 | 1 | } |
2356 | 1 | { |
2357 | 1 | let struct_builder = f2_list_builder.values(); |
2358 | 1 | struct_builder.append(true); |
2359 | 1 | { |
2360 | 1 | let b = struct_builder.field_builder::<BooleanBuilder>(0).unwrap(); |
2361 | 1 | b.append_value(false); |
2362 | 1 | } |
2363 | 1 | { |
2364 | 1 | let b = struct_builder.field_builder::<Float32Builder>(1).unwrap(); |
2365 | 1 | b.append_value(10.2_f32); |
2366 | 1 | } |
2367 | 1 | f2_list_builder.append(true); |
2368 | 1 | } |
2369 | | |
2370 | 1 | let list_array_with_nullable_items = f2_list_builder.finish(); |
2371 | | |
2372 | 1 | let item_field = Arc::new(Field::new( |
2373 | | "item", |
2374 | 1 | list_array_with_nullable_items.values().data_type().clone(), |
2375 | | false, |
2376 | | )); |
2377 | 1 | let list_data_type = DataType::List(item_field); |
2378 | | |
2379 | 1 | let f2_array_data = list_array_with_nullable_items |
2380 | 1 | .to_data() |
2381 | 1 | .into_builder() |
2382 | 1 | .data_type(list_data_type) |
2383 | 1 | .build() |
2384 | 1 | .unwrap(); |
2385 | 1 | let f2_expected = ListArray::from(f2_array_data); |
2386 | | |
2387 | 1 | let mut f3_struct_builder = StructBuilder::new( |
2388 | 1 | vec![Arc::new(Field::new("f3_1", DataType::Utf8, false))], |
2389 | 1 | vec![Box::new(StringBuilder::new()) as Box<dyn ArrayBuilder>], |
2390 | | ); |
2391 | 1 | f3_struct_builder.append(true); |
2392 | 1 | { |
2393 | 1 | let b = f3_struct_builder.field_builder::<StringBuilder>(0).unwrap(); |
2394 | 1 | b.append_value("xyz"); |
2395 | 1 | } |
2396 | 1 | f3_struct_builder.append(false); |
2397 | 1 | { |
2398 | 1 | let b = f3_struct_builder.field_builder::<StringBuilder>(0).unwrap(); |
2399 | 1 | b.append_null(); |
2400 | 1 | } |
2401 | 1 | let f3_expected = f3_struct_builder.finish(); |
2402 | 1 | let f4_fields = [Field::new("f4_1", DataType::Int64, false)]; |
2403 | 1 | let f4_struct_builder = StructBuilder::new( |
2404 | 1 | f4_fields |
2405 | 1 | .iter() |
2406 | 1 | .map(|f| Arc::new(f.clone())) |
2407 | 1 | .collect::<Vec<Arc<Field>>>(), |
2408 | 1 | vec![Box::new(Int64Builder::new()) as Box<dyn arrow_array::builder::ArrayBuilder>], |
2409 | | ); |
2410 | 1 | let mut f4_list_builder = ListBuilder::new(f4_struct_builder); |
2411 | 1 | { |
2412 | 1 | let struct_builder = f4_list_builder.values(); |
2413 | 1 | struct_builder.append(true); |
2414 | 1 | { |
2415 | 1 | let b = struct_builder.field_builder::<Int64Builder>(0).unwrap(); |
2416 | 1 | b.append_value(200); |
2417 | 1 | } |
2418 | 1 | struct_builder.append(false); |
2419 | 1 | { |
2420 | 1 | let b = struct_builder.field_builder::<Int64Builder>(0).unwrap(); |
2421 | 1 | b.append_null(); |
2422 | 1 | } |
2423 | 1 | f4_list_builder.append(true); |
2424 | 1 | } |
2425 | 1 | { |
2426 | 1 | let struct_builder = f4_list_builder.values(); |
2427 | 1 | struct_builder.append(false); |
2428 | 1 | { |
2429 | 1 | let b = struct_builder.field_builder::<Int64Builder>(0).unwrap(); |
2430 | 1 | b.append_null(); |
2431 | 1 | } |
2432 | 1 | struct_builder.append(true); |
2433 | 1 | { |
2434 | 1 | let b = struct_builder.field_builder::<Int64Builder>(0).unwrap(); |
2435 | 1 | b.append_value(300); |
2436 | 1 | } |
2437 | 1 | f4_list_builder.append(true); |
2438 | 1 | } |
2439 | 1 | let f4_expected = f4_list_builder.finish(); |
2440 | | |
2441 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
2442 | 1 | ("f1", Arc::new(f1_expected) as Arc<dyn Array>, false), |
2443 | 1 | ("f2", Arc::new(f2_expected) as Arc<dyn Array>, false), |
2444 | 1 | ("f3", Arc::new(f3_expected) as Arc<dyn Array>, true), |
2445 | 1 | ("f4", Arc::new(f4_expected) as Arc<dyn Array>, false), |
2446 | 1 | ]) |
2447 | 1 | .unwrap(); |
2448 | | |
2449 | 1 | let file = arrow_test_data("avro/nested_records.avro"); |
2450 | 1 | let batch_large = read_file(&file, 8, false); |
2451 | 1 | assert_eq!( |
2452 | | batch_large, expected, |
2453 | 0 | "Decoded RecordBatch does not match expected data for nested records (batch size 8)" |
2454 | | ); |
2455 | 1 | let batch_small = read_file(&file, 3, false); |
2456 | 1 | assert_eq!( |
2457 | | batch_small, expected, |
2458 | 0 | "Decoded RecordBatch does not match expected data for nested records (batch size 3)" |
2459 | | ); |
2460 | 1 | } |
2461 | | |
2462 | | #[test] |
2463 | 1 | fn test_repeated_no_annotation() { |
2464 | 1 | let file = arrow_test_data("avro/repeated_no_annotation.avro"); |
2465 | 1 | let batch_large = read_file(&file, 8, false); |
2466 | | use arrow_array::{Int32Array, Int64Array, ListArray, StringArray, StructArray}; |
2467 | | use arrow_buffer::Buffer; |
2468 | | use arrow_schema::{DataType, Field, Fields}; |
2469 | 1 | let id_array = Int32Array::from(vec![1, 2, 3, 4, 5, 6]); |
2470 | 1 | let number_array = Int64Array::from(vec![ |
2471 | 1 | Some(5555555555), |
2472 | 1 | Some(1111111111), |
2473 | 1 | Some(1111111111), |
2474 | 1 | Some(2222222222), |
2475 | 1 | Some(3333333333), |
2476 | | ]); |
2477 | 1 | let kind_array = |
2478 | 1 | StringArray::from(vec![None, Some("home"), Some("home"), None, Some("mobile")]); |
2479 | 1 | let phone_fields = Fields::from(vec![ |
2480 | 1 | Field::new("number", DataType::Int64, true), |
2481 | 1 | Field::new("kind", DataType::Utf8, true), |
2482 | | ]); |
2483 | 1 | let phone_struct_data = ArrayDataBuilder::new(DataType::Struct(phone_fields)) |
2484 | 1 | .len(5) |
2485 | 1 | .child_data(vec![number_array.into_data(), kind_array.into_data()]) |
2486 | 1 | .build() |
2487 | 1 | .unwrap(); |
2488 | 1 | let phone_struct_array = StructArray::from(phone_struct_data); |
2489 | 1 | let phone_list_offsets = Buffer::from_slice_ref([0, 0, 0, 0, 1, 2, 5]); |
2490 | 1 | let phone_list_validity = Buffer::from_iter([false, false, true, true, true, true]); |
2491 | 1 | let phone_item_field = Field::new("item", phone_struct_array.data_type().clone(), true); |
2492 | 1 | let phone_list_data = ArrayDataBuilder::new(DataType::List(Arc::new(phone_item_field))) |
2493 | 1 | .len(6) |
2494 | 1 | .add_buffer(phone_list_offsets) |
2495 | 1 | .null_bit_buffer(Some(phone_list_validity)) |
2496 | 1 | .child_data(vec![phone_struct_array.into_data()]) |
2497 | 1 | .build() |
2498 | 1 | .unwrap(); |
2499 | 1 | let phone_list_array = ListArray::from(phone_list_data); |
2500 | 1 | let phone_numbers_validity = Buffer::from_iter([false, false, true, true, true, true]); |
2501 | 1 | let phone_numbers_field = Field::new("phone", phone_list_array.data_type().clone(), true); |
2502 | 1 | let phone_numbers_struct_data = |
2503 | 1 | ArrayDataBuilder::new(DataType::Struct(Fields::from(vec![phone_numbers_field]))) |
2504 | 1 | .len(6) |
2505 | 1 | .null_bit_buffer(Some(phone_numbers_validity)) |
2506 | 1 | .child_data(vec![phone_list_array.into_data()]) |
2507 | 1 | .build() |
2508 | 1 | .unwrap(); |
2509 | 1 | let phone_numbers_struct_array = StructArray::from(phone_numbers_struct_data); |
2510 | 1 | let expected = arrow_array::RecordBatch::try_from_iter_with_nullable([ |
2511 | 1 | ("id", Arc::new(id_array) as _, true), |
2512 | 1 | ( |
2513 | 1 | "phoneNumbers", |
2514 | 1 | Arc::new(phone_numbers_struct_array) as _, |
2515 | 1 | true, |
2516 | 1 | ), |
2517 | 1 | ]) |
2518 | 1 | .unwrap(); |
2519 | 1 | assert_eq!(batch_large, expected, "Mismatch for batch_size=8"0 ); |
2520 | 1 | let batch_small = read_file(&file, 3, false); |
2521 | 1 | assert_eq!(batch_small, expected, "Mismatch for batch_size=3"0 ); |
2522 | 1 | } |
2523 | | |
2524 | | #[test] |
2525 | 1 | fn test_nonnullable_impala() { |
2526 | 1 | let file = arrow_test_data("avro/nonnullable.impala.avro"); |
2527 | 1 | let id = Int64Array::from(vec![Some(8)]); |
2528 | 1 | let mut int_array_builder = ListBuilder::new(Int32Builder::new()); |
2529 | 1 | { |
2530 | 1 | let vb = int_array_builder.values(); |
2531 | 1 | vb.append_value(-1); |
2532 | 1 | } |
2533 | 1 | int_array_builder.append(true); // finalize one sub-list |
2534 | 1 | let int_array = int_array_builder.finish(); |
2535 | 1 | let mut iaa_builder = ListBuilder::new(ListBuilder::new(Int32Builder::new())); |
2536 | 1 | { |
2537 | 1 | let inner_list_builder = iaa_builder.values(); |
2538 | 1 | { |
2539 | 1 | let vb = inner_list_builder.values(); |
2540 | 1 | vb.append_value(-1); |
2541 | 1 | vb.append_value(-2); |
2542 | 1 | } |
2543 | 1 | inner_list_builder.append(true); |
2544 | 1 | inner_list_builder.append(true); |
2545 | 1 | } |
2546 | 1 | iaa_builder.append(true); |
2547 | 1 | let int_array_array = iaa_builder.finish(); |
2548 | | use arrow_array::builder::MapFieldNames; |
2549 | 1 | let field_names = MapFieldNames { |
2550 | 1 | entry: "entries".to_string(), |
2551 | 1 | key: "key".to_string(), |
2552 | 1 | value: "value".to_string(), |
2553 | 1 | }; |
2554 | 1 | let mut int_map_builder = |
2555 | 1 | MapBuilder::new(Some(field_names), StringBuilder::new(), Int32Builder::new()); |
2556 | 1 | { |
2557 | 1 | let (keys, vals) = int_map_builder.entries(); |
2558 | 1 | keys.append_value("k1"); |
2559 | 1 | vals.append_value(-1); |
2560 | 1 | } |
2561 | 1 | int_map_builder.append(true).unwrap(); // finalize map for row 0 |
2562 | 1 | let int_map = int_map_builder.finish(); |
2563 | 1 | let field_names2 = MapFieldNames { |
2564 | 1 | entry: "entries".to_string(), |
2565 | 1 | key: "key".to_string(), |
2566 | 1 | value: "value".to_string(), |
2567 | 1 | }; |
2568 | 1 | let mut ima_builder = ListBuilder::new(MapBuilder::new( |
2569 | 1 | Some(field_names2), |
2570 | 1 | StringBuilder::new(), |
2571 | 1 | Int32Builder::new(), |
2572 | | )); |
2573 | 1 | { |
2574 | 1 | let map_builder = ima_builder.values(); |
2575 | 1 | map_builder.append(true).unwrap(); |
2576 | 1 | { |
2577 | 1 | let (keys, vals) = map_builder.entries(); |
2578 | 1 | keys.append_value("k1"); |
2579 | 1 | vals.append_value(1); |
2580 | 1 | } |
2581 | 1 | map_builder.append(true).unwrap(); |
2582 | 1 | map_builder.append(true).unwrap(); |
2583 | 1 | map_builder.append(true).unwrap(); |
2584 | 1 | } |
2585 | 1 | ima_builder.append(true); |
2586 | 1 | let int_map_array_ = ima_builder.finish(); |
2587 | 1 | let mut nested_sb = StructBuilder::new( |
2588 | 1 | vec![ |
2589 | 1 | Arc::new(Field::new("a", DataType::Int32, true)), |
2590 | 1 | Arc::new(Field::new( |
2591 | | "B", |
2592 | 1 | DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), |
2593 | | true, |
2594 | | )), |
2595 | 1 | Arc::new(Field::new( |
2596 | | "c", |
2597 | 1 | DataType::Struct( |
2598 | 1 | vec![Field::new( |
2599 | 1 | "D", |
2600 | 1 | DataType::List(Arc::new(Field::new( |
2601 | 1 | "item", |
2602 | 1 | DataType::List(Arc::new(Field::new( |
2603 | 1 | "item", |
2604 | 1 | DataType::Struct( |
2605 | 1 | vec![ |
2606 | 1 | Field::new("e", DataType::Int32, true), |
2607 | 1 | Field::new("f", DataType::Utf8, true), |
2608 | 1 | ] |
2609 | 1 | .into(), |
2610 | 1 | ), |
2611 | 1 | true, |
2612 | 1 | ))), |
2613 | 1 | true, |
2614 | 1 | ))), |
2615 | 1 | true, |
2616 | 1 | )] |
2617 | 1 | .into(), |
2618 | 1 | ), |
2619 | | true, |
2620 | | )), |
2621 | 1 | Arc::new(Field::new( |
2622 | | "G", |
2623 | 1 | DataType::Map( |
2624 | 1 | Arc::new(Field::new( |
2625 | 1 | "entries", |
2626 | 1 | DataType::Struct( |
2627 | 1 | vec![ |
2628 | 1 | Field::new("key", DataType::Utf8, false), |
2629 | 1 | Field::new( |
2630 | 1 | "value", |
2631 | 1 | DataType::Struct( |
2632 | 1 | vec![Field::new( |
2633 | 1 | "h", |
2634 | 1 | DataType::Struct( |
2635 | 1 | vec![Field::new( |
2636 | 1 | "i", |
2637 | 1 | DataType::List(Arc::new(Field::new( |
2638 | 1 | "item", |
2639 | 1 | DataType::Float64, |
2640 | 1 | true, |
2641 | 1 | ))), |
2642 | 1 | true, |
2643 | 1 | )] |
2644 | 1 | .into(), |
2645 | 1 | ), |
2646 | 1 | true, |
2647 | 1 | )] |
2648 | 1 | .into(), |
2649 | 1 | ), |
2650 | 1 | true, |
2651 | 1 | ), |
2652 | 1 | ] |
2653 | 1 | .into(), |
2654 | 1 | ), |
2655 | 1 | false, |
2656 | 1 | )), |
2657 | 1 | false, |
2658 | 1 | ), |
2659 | | true, |
2660 | | )), |
2661 | | ], |
2662 | 1 | vec![ |
2663 | 1 | Box::new(Int32Builder::new()), |
2664 | 1 | Box::new(ListBuilder::new(Int32Builder::new())), |
2665 | | { |
2666 | 1 | let d_field = Field::new( |
2667 | | "D", |
2668 | 1 | DataType::List(Arc::new(Field::new( |
2669 | 1 | "item", |
2670 | 1 | DataType::List(Arc::new(Field::new( |
2671 | 1 | "item", |
2672 | 1 | DataType::Struct( |
2673 | 1 | vec![ |
2674 | 1 | Field::new("e", DataType::Int32, true), |
2675 | 1 | Field::new("f", DataType::Utf8, true), |
2676 | 1 | ] |
2677 | 1 | .into(), |
2678 | 1 | ), |
2679 | 1 | true, |
2680 | 1 | ))), |
2681 | 1 | true, |
2682 | 1 | ))), |
2683 | | true, |
2684 | | ); |
2685 | 1 | Box::new(StructBuilder::new( |
2686 | 1 | vec![Arc::new(d_field)], |
2687 | 1 | vec![Box::new({ |
2688 | 1 | let ef_struct_builder = StructBuilder::new( |
2689 | 1 | vec![ |
2690 | 1 | Arc::new(Field::new("e", DataType::Int32, true)), |
2691 | 1 | Arc::new(Field::new("f", DataType::Utf8, true)), |
2692 | 1 | ], |
2693 | 1 | vec![ |
2694 | 1 | Box::new(Int32Builder::new()), |
2695 | 1 | Box::new(StringBuilder::new()), |
2696 | 1 | ], |
2697 | 1 | ); |
2698 | 1 | let list_of_ef = ListBuilder::new(ef_struct_builder); |
2699 | 1 | ListBuilder::new(list_of_ef) |
2700 | 1 | })], |
2701 | 1 | )) |
2702 | | }, |
2703 | | { |
2704 | 1 | let map_field_names = MapFieldNames { |
2705 | 1 | entry: "entries".to_string(), |
2706 | 1 | key: "key".to_string(), |
2707 | 1 | value: "value".to_string(), |
2708 | 1 | }; |
2709 | 1 | let i_list_builder = ListBuilder::new(Float64Builder::new()); |
2710 | 1 | let h_struct = StructBuilder::new( |
2711 | 1 | vec![Arc::new(Field::new( |
2712 | | "i", |
2713 | 1 | DataType::List(Arc::new(Field::new("item", DataType::Float64, true))), |
2714 | | true, |
2715 | | ))], |
2716 | 1 | vec![Box::new(i_list_builder)], |
2717 | | ); |
2718 | 1 | let g_value_builder = StructBuilder::new( |
2719 | 1 | vec![Arc::new(Field::new( |
2720 | | "h", |
2721 | 1 | DataType::Struct( |
2722 | 1 | vec![Field::new( |
2723 | 1 | "i", |
2724 | 1 | DataType::List(Arc::new(Field::new( |
2725 | 1 | "item", |
2726 | 1 | DataType::Float64, |
2727 | 1 | true, |
2728 | 1 | ))), |
2729 | 1 | true, |
2730 | 1 | )] |
2731 | 1 | .into(), |
2732 | 1 | ), |
2733 | | true, |
2734 | | ))], |
2735 | 1 | vec![Box::new(h_struct)], |
2736 | | ); |
2737 | 1 | Box::new(MapBuilder::new( |
2738 | 1 | Some(map_field_names), |
2739 | 1 | StringBuilder::new(), |
2740 | 1 | g_value_builder, |
2741 | 1 | )) |
2742 | | }, |
2743 | | ], |
2744 | | ); |
2745 | 1 | nested_sb.append(true); |
2746 | 1 | { |
2747 | 1 | let a_builder = nested_sb.field_builder::<Int32Builder>(0).unwrap(); |
2748 | 1 | a_builder.append_value(-1); |
2749 | 1 | } |
2750 | 1 | { |
2751 | 1 | let b_builder = nested_sb |
2752 | 1 | .field_builder::<ListBuilder<Int32Builder>>(1) |
2753 | 1 | .unwrap(); |
2754 | 1 | { |
2755 | 1 | let vb = b_builder.values(); |
2756 | 1 | vb.append_value(-1); |
2757 | 1 | } |
2758 | 1 | b_builder.append(true); |
2759 | 1 | } |
2760 | | { |
2761 | 1 | let c_struct_builder = nested_sb.field_builder::<StructBuilder>(2).unwrap(); |
2762 | 1 | c_struct_builder.append(true); |
2763 | 1 | let d_list_builder = c_struct_builder |
2764 | 1 | .field_builder::<ListBuilder<ListBuilder<StructBuilder>>>(0) |
2765 | 1 | .unwrap(); |
2766 | 1 | { |
2767 | 1 | let sub_list_builder = d_list_builder.values(); |
2768 | 1 | { |
2769 | 1 | let ef_struct = sub_list_builder.values(); |
2770 | 1 | ef_struct.append(true); |
2771 | 1 | { |
2772 | 1 | let e_b = ef_struct.field_builder::<Int32Builder>(0).unwrap(); |
2773 | 1 | e_b.append_value(-1); |
2774 | 1 | let f_b = ef_struct.field_builder::<StringBuilder>(1).unwrap(); |
2775 | 1 | f_b.append_value("nonnullable"); |
2776 | 1 | } |
2777 | 1 | sub_list_builder.append(true); |
2778 | 1 | } |
2779 | 1 | d_list_builder.append(true); |
2780 | 1 | } |
2781 | | } |
2782 | 1 | { |
2783 | 1 | let g_map_builder = nested_sb |
2784 | 1 | .field_builder::<MapBuilder<StringBuilder, StructBuilder>>(3) |
2785 | 1 | .unwrap(); |
2786 | 1 | g_map_builder.append(true).unwrap(); |
2787 | 1 | } |
2788 | 1 | let nested_struct = nested_sb.finish(); |
2789 | 1 | let expected = RecordBatch::try_from_iter_with_nullable([ |
2790 | 1 | ("ID", Arc::new(id) as Arc<dyn Array>, true), |
2791 | 1 | ("Int_Array", Arc::new(int_array), true), |
2792 | 1 | ("int_array_array", Arc::new(int_array_array), true), |
2793 | 1 | ("Int_Map", Arc::new(int_map), true), |
2794 | 1 | ("int_map_array", Arc::new(int_map_array_), true), |
2795 | 1 | ("nested_Struct", Arc::new(nested_struct), true), |
2796 | 1 | ]) |
2797 | 1 | .unwrap(); |
2798 | 1 | let batch_large = read_file(&file, 8, false); |
2799 | 1 | assert_eq!(batch_large, expected, "Mismatch for batch_size=8"0 ); |
2800 | 1 | let batch_small = read_file(&file, 3, false); |
2801 | 1 | assert_eq!(batch_small, expected, "Mismatch for batch_size=3"0 ); |
2802 | 1 | } |
2803 | | |
2804 | | #[test] |
2805 | 1 | fn test_nonnullable_impala_strict() { |
2806 | 1 | let file = arrow_test_data("avro/nonnullable.impala.avro"); |
2807 | 1 | let err = read_file_strict(&file, 8, false).unwrap_err(); |
2808 | 1 | assert!(err.to_string().contains( |
2809 | 1 | "Found Avro union of the form ['T','null'], which is disallowed in strict_mode" |
2810 | 1 | )); |
2811 | 1 | } |
2812 | | |
2813 | | #[test] |
2814 | 1 | fn test_nullable_impala() { |
2815 | 1 | let file = arrow_test_data("avro/nullable.impala.avro"); |
2816 | 1 | let batch1 = read_file(&file, 3, false); |
2817 | 1 | let batch2 = read_file(&file, 8, false); |
2818 | 1 | assert_eq!(batch1, batch2); |
2819 | 1 | let batch = batch1; |
2820 | 1 | assert_eq!(batch.num_rows(), 7); |
2821 | 1 | let id_array = batch |
2822 | 1 | .column(0) |
2823 | 1 | .as_any() |
2824 | 1 | .downcast_ref::<Int64Array>() |
2825 | 1 | .expect("id column should be an Int64Array"); |
2826 | 1 | let expected_ids = [1, 2, 3, 4, 5, 6, 7]; |
2827 | 7 | for (i, &expected_id) in expected_ids1 .iter1 ().enumerate1 () { |
2828 | 7 | assert_eq!(id_array.value(i), expected_id, "Mismatch in id at row {i}"0 ,); |
2829 | | } |
2830 | 1 | let int_array = batch |
2831 | 1 | .column(1) |
2832 | 1 | .as_any() |
2833 | 1 | .downcast_ref::<ListArray>() |
2834 | 1 | .expect("int_array column should be a ListArray"); |
2835 | | { |
2836 | 1 | let offsets = int_array.value_offsets(); |
2837 | 1 | let start = offsets[0] as usize; |
2838 | 1 | let end = offsets[1] as usize; |
2839 | 1 | let values = int_array |
2840 | 1 | .values() |
2841 | 1 | .as_any() |
2842 | 1 | .downcast_ref::<Int32Array>() |
2843 | 1 | .expect("Values of int_array should be an Int32Array"); |
2844 | 3 | let row01 : Vec<Option<i32>>1 = (start..end)1 .map1 (|i| Some(values.value(i))).collect1 (); |
2845 | 1 | assert_eq!( |
2846 | | row0, |
2847 | 1 | vec![Some(1), Some(2), Some(3)], |
2848 | 0 | "Mismatch in int_array row 0" |
2849 | | ); |
2850 | | } |
2851 | 1 | let nested_struct = batch |
2852 | 1 | .column(5) |
2853 | 1 | .as_any() |
2854 | 1 | .downcast_ref::<StructArray>() |
2855 | 1 | .expect("nested_struct column should be a StructArray"); |
2856 | 1 | let a_array = nested_struct |
2857 | 1 | .column_by_name("A") |
2858 | 1 | .expect("Field A should exist in nested_struct") |
2859 | 1 | .as_any() |
2860 | 1 | .downcast_ref::<Int32Array>() |
2861 | 1 | .expect("Field A should be an Int32Array"); |
2862 | 1 | assert_eq!(a_array.value(0), 1, "Mismatch in nested_struct.A at row 0"0 ); |
2863 | 1 | assert!( |
2864 | 1 | !a_array.is_valid(1), |
2865 | 0 | "Expected null in nested_struct.A at row 1" |
2866 | | ); |
2867 | 1 | assert!( |
2868 | 1 | !a_array.is_valid(3), |
2869 | 0 | "Expected null in nested_struct.A at row 3" |
2870 | | ); |
2871 | 1 | assert_eq!(a_array.value(6), 7, "Mismatch in nested_struct.A at row 6"0 ); |
2872 | 1 | } |
2873 | | |
2874 | | #[test] |
2875 | 1 | fn test_nullable_impala_strict() { |
2876 | 1 | let file = arrow_test_data("avro/nullable.impala.avro"); |
2877 | 1 | let err = read_file_strict(&file, 8, false).unwrap_err(); |
2878 | 1 | assert!(err.to_string().contains( |
2879 | 1 | "Found Avro union of the form ['T','null'], which is disallowed in strict_mode" |
2880 | 1 | )); |
2881 | 1 | } |
2882 | | |
2883 | | #[test] |
2884 | 1 | fn test_nested_record_type_reuse() { |
2885 | | // The .avro file has the following schema: |
2886 | | // { |
2887 | | // "type" : "record", |
2888 | | // "name" : "Record", |
2889 | | // "fields" : [ { |
2890 | | // "name" : "nested", |
2891 | | // "type" : { |
2892 | | // "type" : "record", |
2893 | | // "name" : "Nested", |
2894 | | // "fields" : [ { |
2895 | | // "name" : "nested_int", |
2896 | | // "type" : "int" |
2897 | | // } ] |
2898 | | // } |
2899 | | // }, { |
2900 | | // "name" : "nestedRecord", |
2901 | | // "type" : "Nested" |
2902 | | // }, { |
2903 | | // "name" : "nestedArray", |
2904 | | // "type" : { |
2905 | | // "type" : "array", |
2906 | | // "items" : "Nested" |
2907 | | // } |
2908 | | // } ] |
2909 | | // } |
2910 | 1 | let batch = read_file("test/data/nested_record_reuse.avro", 8, false); |
2911 | 1 | let schema = batch.schema(); |
2912 | | |
2913 | | // Verify schema structure |
2914 | 1 | assert_eq!(schema.fields().len(), 3); |
2915 | 1 | let fields = schema.fields(); |
2916 | 1 | assert_eq!(fields[0].name(), "nested"); |
2917 | 1 | assert_eq!(fields[1].name(), "nestedRecord"); |
2918 | 1 | assert_eq!(fields[2].name(), "nestedArray"); |
2919 | 1 | assert!(matches!0 (fields[0].data_type(), DataType::Struct(_))); |
2920 | 1 | assert!(matches!0 (fields[1].data_type(), DataType::Struct(_))); |
2921 | 1 | assert!(matches!0 (fields[2].data_type(), DataType::List(_))); |
2922 | | |
2923 | | // Validate that the nested record type |
2924 | 1 | if let DataType::Struct(nested_fields) = fields[0].data_type() { |
2925 | 1 | assert_eq!(nested_fields.len(), 1); |
2926 | 1 | assert_eq!(nested_fields[0].name(), "nested_int"); |
2927 | 1 | assert_eq!(nested_fields[0].data_type(), &DataType::Int32); |
2928 | 0 | } |
2929 | | |
2930 | | // Validate that the nested record type is reused |
2931 | 1 | assert_eq!(fields[0].data_type(), fields[1].data_type()); |
2932 | 1 | if let DataType::List(array_field) = fields[2].data_type() { |
2933 | 1 | assert_eq!(array_field.data_type(), fields[0].data_type()); |
2934 | 0 | } |
2935 | | |
2936 | | // Validate data |
2937 | 1 | assert_eq!(batch.num_rows(), 2); |
2938 | 1 | assert_eq!(batch.num_columns(), 3); |
2939 | | |
2940 | | // Validate the first column (nested) |
2941 | 1 | let nested_col = batch |
2942 | 1 | .column(0) |
2943 | 1 | .as_any() |
2944 | 1 | .downcast_ref::<StructArray>() |
2945 | 1 | .unwrap(); |
2946 | 1 | let nested_int_array = nested_col |
2947 | 1 | .column_by_name("nested_int") |
2948 | 1 | .unwrap() |
2949 | 1 | .as_any() |
2950 | 1 | .downcast_ref::<Int32Array>() |
2951 | 1 | .unwrap(); |
2952 | 1 | assert_eq!(nested_int_array.value(0), 42); |
2953 | 1 | assert_eq!(nested_int_array.value(1), 99); |
2954 | | |
2955 | | // Validate the second column (nestedRecord) |
2956 | 1 | let nested_record_col = batch |
2957 | 1 | .column(1) |
2958 | 1 | .as_any() |
2959 | 1 | .downcast_ref::<StructArray>() |
2960 | 1 | .unwrap(); |
2961 | 1 | let nested_record_int_array = nested_record_col |
2962 | 1 | .column_by_name("nested_int") |
2963 | 1 | .unwrap() |
2964 | 1 | .as_any() |
2965 | 1 | .downcast_ref::<Int32Array>() |
2966 | 1 | .unwrap(); |
2967 | 1 | assert_eq!(nested_record_int_array.value(0), 100); |
2968 | 1 | assert_eq!(nested_record_int_array.value(1), 200); |
2969 | | |
2970 | | // Validate the third column (nestedArray) |
2971 | 1 | let nested_array_col = batch |
2972 | 1 | .column(2) |
2973 | 1 | .as_any() |
2974 | 1 | .downcast_ref::<ListArray>() |
2975 | 1 | .unwrap(); |
2976 | 1 | assert_eq!(nested_array_col.len(), 2); |
2977 | 1 | let first_array_struct = nested_array_col.value(0); |
2978 | 1 | let first_array_struct_array = first_array_struct |
2979 | 1 | .as_any() |
2980 | 1 | .downcast_ref::<StructArray>() |
2981 | 1 | .unwrap(); |
2982 | 1 | let first_array_int_values = first_array_struct_array |
2983 | 1 | .column_by_name("nested_int") |
2984 | 1 | .unwrap() |
2985 | 1 | .as_any() |
2986 | 1 | .downcast_ref::<Int32Array>() |
2987 | 1 | .unwrap(); |
2988 | 1 | assert_eq!(first_array_int_values.len(), 3); |
2989 | 1 | assert_eq!(first_array_int_values.value(0), 1); |
2990 | 1 | assert_eq!(first_array_int_values.value(1), 2); |
2991 | 1 | assert_eq!(first_array_int_values.value(2), 3); |
2992 | 1 | } |
2993 | | |
2994 | | #[test] |
2995 | 1 | fn test_enum_type_reuse() { |
2996 | | // The .avro file has the following schema: |
2997 | | // { |
2998 | | // "type" : "record", |
2999 | | // "name" : "Record", |
3000 | | // "fields" : [ { |
3001 | | // "name" : "status", |
3002 | | // "type" : { |
3003 | | // "type" : "enum", |
3004 | | // "name" : "Status", |
3005 | | // "symbols" : [ "ACTIVE", "INACTIVE", "PENDING" ] |
3006 | | // } |
3007 | | // }, { |
3008 | | // "name" : "backupStatus", |
3009 | | // "type" : "Status" |
3010 | | // }, { |
3011 | | // "name" : "statusHistory", |
3012 | | // "type" : { |
3013 | | // "type" : "array", |
3014 | | // "items" : "Status" |
3015 | | // } |
3016 | | // } ] |
3017 | | // } |
3018 | 1 | let batch = read_file("test/data/enum_reuse.avro", 8, false); |
3019 | 1 | let schema = batch.schema(); |
3020 | | |
3021 | | // Verify schema structure |
3022 | 1 | assert_eq!(schema.fields().len(), 3); |
3023 | 1 | let fields = schema.fields(); |
3024 | 1 | assert_eq!(fields[0].name(), "status"); |
3025 | 1 | assert_eq!(fields[1].name(), "backupStatus"); |
3026 | 1 | assert_eq!(fields[2].name(), "statusHistory"); |
3027 | 1 | assert!(matches!0 (fields[0].data_type(), DataType::Dictionary(_, _))); |
3028 | 1 | assert!(matches!0 (fields[1].data_type(), DataType::Dictionary(_, _))); |
3029 | 1 | assert!(matches!0 (fields[2].data_type(), DataType::List(_))); |
3030 | | |
3031 | 1 | if let DataType::Dictionary(key_type, value_type) = fields[0].data_type() { |
3032 | 1 | assert_eq!(key_type.as_ref(), &DataType::Int32); |
3033 | 1 | assert_eq!(value_type.as_ref(), &DataType::Utf8); |
3034 | 0 | } |
3035 | | |
3036 | | // Validate that the enum types are reused |
3037 | 1 | assert_eq!(fields[0].data_type(), fields[1].data_type()); |
3038 | 1 | if let DataType::List(array_field) = fields[2].data_type() { |
3039 | 1 | assert_eq!(array_field.data_type(), fields[0].data_type()); |
3040 | 0 | } |
3041 | | |
3042 | | // Validate data - should have 2 rows |
3043 | 1 | assert_eq!(batch.num_rows(), 2); |
3044 | 1 | assert_eq!(batch.num_columns(), 3); |
3045 | | |
3046 | | // Get status enum values |
3047 | 1 | let status_col = batch |
3048 | 1 | .column(0) |
3049 | 1 | .as_any() |
3050 | 1 | .downcast_ref::<DictionaryArray<Int32Type>>() |
3051 | 1 | .unwrap(); |
3052 | 1 | let status_values = status_col |
3053 | 1 | .values() |
3054 | 1 | .as_any() |
3055 | 1 | .downcast_ref::<StringArray>() |
3056 | 1 | .unwrap(); |
3057 | | |
3058 | | // First row should be "ACTIVE", second row should be "PENDING" |
3059 | 1 | assert_eq!( |
3060 | 1 | status_values.value(status_col.key(0).unwrap() as usize), |
3061 | | "ACTIVE" |
3062 | | ); |
3063 | 1 | assert_eq!( |
3064 | 1 | status_values.value(status_col.key(1).unwrap() as usize), |
3065 | | "PENDING" |
3066 | | ); |
3067 | | |
3068 | | // Get backupStatus enum values (same as status) |
3069 | 1 | let backup_status_col = batch |
3070 | 1 | .column(1) |
3071 | 1 | .as_any() |
3072 | 1 | .downcast_ref::<DictionaryArray<Int32Type>>() |
3073 | 1 | .unwrap(); |
3074 | 1 | let backup_status_values = backup_status_col |
3075 | 1 | .values() |
3076 | 1 | .as_any() |
3077 | 1 | .downcast_ref::<StringArray>() |
3078 | 1 | .unwrap(); |
3079 | | |
3080 | | // First row should be "INACTIVE", second row should be "ACTIVE" |
3081 | 1 | assert_eq!( |
3082 | 1 | backup_status_values.value(backup_status_col.key(0).unwrap() as usize), |
3083 | | "INACTIVE" |
3084 | | ); |
3085 | 1 | assert_eq!( |
3086 | 1 | backup_status_values.value(backup_status_col.key(1).unwrap() as usize), |
3087 | | "ACTIVE" |
3088 | | ); |
3089 | | |
3090 | | // Get statusHistory array |
3091 | 1 | let status_history_col = batch |
3092 | 1 | .column(2) |
3093 | 1 | .as_any() |
3094 | 1 | .downcast_ref::<ListArray>() |
3095 | 1 | .unwrap(); |
3096 | 1 | assert_eq!(status_history_col.len(), 2); |
3097 | | |
3098 | | // Validate first row's array data |
3099 | 1 | let first_array_dict = status_history_col.value(0); |
3100 | 1 | let first_array_dict_array = first_array_dict |
3101 | 1 | .as_any() |
3102 | 1 | .downcast_ref::<DictionaryArray<Int32Type>>() |
3103 | 1 | .unwrap(); |
3104 | 1 | let first_array_values = first_array_dict_array |
3105 | 1 | .values() |
3106 | 1 | .as_any() |
3107 | 1 | .downcast_ref::<StringArray>() |
3108 | 1 | .unwrap(); |
3109 | | |
3110 | | // First row: ["PENDING", "ACTIVE", "INACTIVE"] |
3111 | 1 | assert_eq!(first_array_dict_array.len(), 3); |
3112 | 1 | assert_eq!( |
3113 | 1 | first_array_values.value(first_array_dict_array.key(0).unwrap() as usize), |
3114 | | "PENDING" |
3115 | | ); |
3116 | 1 | assert_eq!( |
3117 | 1 | first_array_values.value(first_array_dict_array.key(1).unwrap() as usize), |
3118 | | "ACTIVE" |
3119 | | ); |
3120 | 1 | assert_eq!( |
3121 | 1 | first_array_values.value(first_array_dict_array.key(2).unwrap() as usize), |
3122 | | "INACTIVE" |
3123 | | ); |
3124 | 1 | } |
3125 | | } |