arrow_ipc/
reader.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Arrow IPC File and Stream Readers
19//!
20//! # Notes
21//!
22//! The [`FileReader`] and [`StreamReader`] have similar interfaces,
23//! however the [`FileReader`] expects a reader that supports [`Seek`]ing
24//!
25//! [`Seek`]: std::io::Seek
26
27mod stream;
28
29pub use stream::*;
30
31use flatbuffers::{VectorIter, VerifierOptions};
32use std::collections::{HashMap, VecDeque};
33use std::fmt;
34use std::io::{BufReader, Read, Seek, SeekFrom};
35use std::sync::Arc;
36
37use arrow_array::*;
38use arrow_buffer::{ArrowNativeType, BooleanBuffer, Buffer, MutableBuffer, ScalarBuffer};
39use arrow_data::{ArrayData, ArrayDataBuilder, UnsafeFlag};
40use arrow_schema::*;
41
42use crate::compression::CompressionCodec;
43use crate::{Block, FieldNode, Message, MetadataVersion, CONTINUATION_MARKER};
44use DataType::*;
45
46/// Read a buffer based on offset and length
47/// From <https://github.com/apache/arrow/blob/6a936c4ff5007045e86f65f1a6b6c3c955ad5103/format/Message.fbs#L58>
48/// Each constituent buffer is first compressed with the indicated
49/// compressor, and then written with the uncompressed length in the first 8
50/// bytes as a 64-bit little-endian signed integer followed by the compressed
51/// buffer bytes (and then padding as required by the protocol). The
52/// uncompressed length may be set to -1 to indicate that the data that
53/// follows is not compressed, which can be useful for cases where
54/// compression does not yield appreciable savings.
55fn read_buffer(
56    buf: &crate::Buffer,
57    a_data: &Buffer,
58    compression_codec: Option<CompressionCodec>,
59) -> Result<Buffer, ArrowError> {
60    let start_offset = buf.offset() as usize;
61    let buf_data = a_data.slice_with_length(start_offset, buf.length() as usize);
62    // corner case: empty buffer
63    match (buf_data.is_empty(), compression_codec) {
64        (true, _) | (_, None) => Ok(buf_data),
65        (false, Some(decompressor)) => decompressor.decompress_to_buffer(&buf_data),
66    }
67}
68impl RecordBatchDecoder<'_> {
69    /// Coordinates reading arrays based on data types.
70    ///
71    /// `variadic_counts` encodes the number of buffers to read for variadic types (e.g., Utf8View, BinaryView)
72    /// When encounter such types, we pop from the front of the queue to get the number of buffers to read.
73    ///
74    /// Notes:
75    /// * In the IPC format, null buffers are always set, but may be empty. We discard them if an array has 0 nulls
76    /// * Numeric values inside list arrays are often stored as 64-bit values regardless of their data type size.
77    ///   We thus:
78    ///     - check if the bit width of non-64-bit numbers is 64, and
79    ///     - read the buffer as 64-bit (signed integer or float), and
80    ///     - cast the 64-bit array to the appropriate data type
81    fn create_array(
82        &mut self,
83        field: &Field,
84        variadic_counts: &mut VecDeque<i64>,
85    ) -> Result<ArrayRef, ArrowError> {
86        let data_type = field.data_type();
87        match data_type {
88            Utf8 | Binary | LargeBinary | LargeUtf8 => {
89                let field_node = self.next_node(field)?;
90                let buffers = [
91                    self.next_buffer()?,
92                    self.next_buffer()?,
93                    self.next_buffer()?,
94                ];
95                self.create_primitive_array(field_node, data_type, &buffers)
96            }
97            BinaryView | Utf8View => {
98                let count = variadic_counts
99                    .pop_front()
100                    .ok_or(ArrowError::IpcError(format!(
101                        "Missing variadic count for {data_type} column"
102                    )))?;
103                let count = count + 2; // view and null buffer.
104                let buffers = (0..count)
105                    .map(|_| self.next_buffer())
106                    .collect::<Result<Vec<_>, _>>()?;
107                let field_node = self.next_node(field)?;
108                self.create_primitive_array(field_node, data_type, &buffers)
109            }
110            FixedSizeBinary(_) => {
111                let field_node = self.next_node(field)?;
112                let buffers = [self.next_buffer()?, self.next_buffer()?];
113                self.create_primitive_array(field_node, data_type, &buffers)
114            }
115            List(ref list_field) | LargeList(ref list_field) | Map(ref list_field, _) => {
116                let list_node = self.next_node(field)?;
117                let list_buffers = [self.next_buffer()?, self.next_buffer()?];
118                let values = self.create_array(list_field, variadic_counts)?;
119                self.create_list_array(list_node, data_type, &list_buffers, values)
120            }
121            FixedSizeList(ref list_field, _) => {
122                let list_node = self.next_node(field)?;
123                let list_buffers = [self.next_buffer()?];
124                let values = self.create_array(list_field, variadic_counts)?;
125                self.create_list_array(list_node, data_type, &list_buffers, values)
126            }
127            Struct(struct_fields) => {
128                let struct_node = self.next_node(field)?;
129                let null_buffer = self.next_buffer()?;
130
131                // read the arrays for each field
132                let mut struct_arrays = vec![];
133                // TODO investigate whether just knowing the number of buffers could
134                // still work
135                for struct_field in struct_fields {
136                    let child = self.create_array(struct_field, variadic_counts)?;
137                    struct_arrays.push(child);
138                }
139                self.create_struct_array(struct_node, null_buffer, struct_fields, struct_arrays)
140            }
141            RunEndEncoded(run_ends_field, values_field) => {
142                let run_node = self.next_node(field)?;
143                let run_ends = self.create_array(run_ends_field, variadic_counts)?;
144                let values = self.create_array(values_field, variadic_counts)?;
145
146                let run_array_length = run_node.length() as usize;
147                let builder = ArrayData::builder(data_type.clone())
148                    .len(run_array_length)
149                    .offset(0)
150                    .add_child_data(run_ends.into_data())
151                    .add_child_data(values.into_data());
152                self.create_array_from_builder(builder)
153            }
154            // Create dictionary array from RecordBatch
155            Dictionary(_, _) => {
156                let index_node = self.next_node(field)?;
157                let index_buffers = [self.next_buffer()?, self.next_buffer()?];
158
159                #[allow(deprecated)]
160                let dict_id = field.dict_id().ok_or_else(|| {
161                    ArrowError::ParseError(format!("Field {field} does not have dict id"))
162                })?;
163
164                let value_array = self.dictionaries_by_id.get(&dict_id).ok_or_else(|| {
165                    ArrowError::ParseError(format!(
166                        "Cannot find a dictionary batch with dict id: {dict_id}"
167                    ))
168                })?;
169
170                self.create_dictionary_array(
171                    index_node,
172                    data_type,
173                    &index_buffers,
174                    value_array.clone(),
175                )
176            }
177            Union(fields, mode) => {
178                let union_node = self.next_node(field)?;
179                let len = union_node.length() as usize;
180
181                // In V4, union types has validity bitmap
182                // In V5 and later, union types have no validity bitmap
183                if self.version < MetadataVersion::V5 {
184                    self.next_buffer()?;
185                }
186
187                let type_ids: ScalarBuffer<i8> =
188                    self.next_buffer()?.slice_with_length(0, len).into();
189
190                let value_offsets = match mode {
191                    UnionMode::Dense => {
192                        let offsets: ScalarBuffer<i32> =
193                            self.next_buffer()?.slice_with_length(0, len * 4).into();
194                        Some(offsets)
195                    }
196                    UnionMode::Sparse => None,
197                };
198
199                let mut children = Vec::with_capacity(fields.len());
200
201                for (_id, field) in fields.iter() {
202                    let child = self.create_array(field, variadic_counts)?;
203                    children.push(child);
204                }
205
206                let array = if self.skip_validation.get() {
207                    // safety: flag can only be set via unsafe code
208                    unsafe {
209                        UnionArray::new_unchecked(fields.clone(), type_ids, value_offsets, children)
210                    }
211                } else {
212                    UnionArray::try_new(fields.clone(), type_ids, value_offsets, children)?
213                };
214                Ok(Arc::new(array))
215            }
216            Null => {
217                let node = self.next_node(field)?;
218                let length = node.length();
219                let null_count = node.null_count();
220
221                if length != null_count {
222                    return Err(ArrowError::SchemaError(format!(
223                        "Field {field} of NullArray has unequal null_count {null_count} and len {length}"
224                    )));
225                }
226
227                let builder = ArrayData::builder(data_type.clone())
228                    .len(length as usize)
229                    .offset(0);
230                self.create_array_from_builder(builder)
231            }
232            _ => {
233                let field_node = self.next_node(field)?;
234                let buffers = [self.next_buffer()?, self.next_buffer()?];
235                self.create_primitive_array(field_node, data_type, &buffers)
236            }
237        }
238    }
239
240    /// Reads the correct number of buffers based on data type and null_count, and creates a
241    /// primitive array ref
242    fn create_primitive_array(
243        &self,
244        field_node: &FieldNode,
245        data_type: &DataType,
246        buffers: &[Buffer],
247    ) -> Result<ArrayRef, ArrowError> {
248        let length = field_node.length() as usize;
249        let null_buffer = (field_node.null_count() > 0).then_some(buffers[0].clone());
250        let builder = match data_type {
251            Utf8 | Binary | LargeBinary | LargeUtf8 => {
252                // read 3 buffers: null buffer (optional), offsets buffer and data buffer
253                ArrayData::builder(data_type.clone())
254                    .len(length)
255                    .buffers(buffers[1..3].to_vec())
256                    .null_bit_buffer(null_buffer)
257            }
258            BinaryView | Utf8View => ArrayData::builder(data_type.clone())
259                .len(length)
260                .buffers(buffers[1..].to_vec())
261                .null_bit_buffer(null_buffer),
262            _ if data_type.is_primitive() || matches!(data_type, Boolean | FixedSizeBinary(_)) => {
263                // read 2 buffers: null buffer (optional) and data buffer
264                ArrayData::builder(data_type.clone())
265                    .len(length)
266                    .add_buffer(buffers[1].clone())
267                    .null_bit_buffer(null_buffer)
268            }
269            t => unreachable!("Data type {:?} either unsupported or not primitive", t),
270        };
271
272        self.create_array_from_builder(builder)
273    }
274
275    /// Update the ArrayDataBuilder based on settings in this decoder
276    fn create_array_from_builder(&self, builder: ArrayDataBuilder) -> Result<ArrayRef, ArrowError> {
277        let mut builder = builder.align_buffers(!self.require_alignment);
278        if self.skip_validation.get() {
279            // SAFETY: flag can only be set via unsafe code
280            unsafe { builder = builder.skip_validation(true) }
281        };
282        Ok(make_array(builder.build()?))
283    }
284
285    /// Reads the correct number of buffers based on list type and null_count, and creates a
286    /// list array ref
287    fn create_list_array(
288        &self,
289        field_node: &FieldNode,
290        data_type: &DataType,
291        buffers: &[Buffer],
292        child_array: ArrayRef,
293    ) -> Result<ArrayRef, ArrowError> {
294        let null_buffer = (field_node.null_count() > 0).then_some(buffers[0].clone());
295        let length = field_node.length() as usize;
296        let child_data = child_array.into_data();
297        let builder = match data_type {
298            List(_) | LargeList(_) | Map(_, _) => ArrayData::builder(data_type.clone())
299                .len(length)
300                .add_buffer(buffers[1].clone())
301                .add_child_data(child_data)
302                .null_bit_buffer(null_buffer),
303
304            FixedSizeList(_, _) => ArrayData::builder(data_type.clone())
305                .len(length)
306                .add_child_data(child_data)
307                .null_bit_buffer(null_buffer),
308
309            _ => unreachable!("Cannot create list or map array from {:?}", data_type),
310        };
311
312        self.create_array_from_builder(builder)
313    }
314
315    fn create_struct_array(
316        &self,
317        struct_node: &FieldNode,
318        null_buffer: Buffer,
319        struct_fields: &Fields,
320        struct_arrays: Vec<ArrayRef>,
321    ) -> Result<ArrayRef, ArrowError> {
322        let null_count = struct_node.null_count() as usize;
323        let len = struct_node.length() as usize;
324
325        let nulls = (null_count > 0).then(|| BooleanBuffer::new(null_buffer, 0, len).into());
326        if struct_arrays.is_empty() {
327            // `StructArray::from` can't infer the correct row count
328            // if we have zero fields
329            return Ok(Arc::new(StructArray::new_empty_fields(len, nulls)));
330        }
331
332        let struct_array = if self.skip_validation.get() {
333            // safety: flag can only be set via unsafe code
334            unsafe { StructArray::new_unchecked(struct_fields.clone(), struct_arrays, nulls) }
335        } else {
336            StructArray::try_new(struct_fields.clone(), struct_arrays, nulls)?
337        };
338
339        Ok(Arc::new(struct_array))
340    }
341
342    /// Reads the correct number of buffers based on list type and null_count, and creates a
343    /// list array ref
344    fn create_dictionary_array(
345        &self,
346        field_node: &FieldNode,
347        data_type: &DataType,
348        buffers: &[Buffer],
349        value_array: ArrayRef,
350    ) -> Result<ArrayRef, ArrowError> {
351        if let Dictionary(_, _) = *data_type {
352            let null_buffer = (field_node.null_count() > 0).then_some(buffers[0].clone());
353            let builder = ArrayData::builder(data_type.clone())
354                .len(field_node.length() as usize)
355                .add_buffer(buffers[1].clone())
356                .add_child_data(value_array.into_data())
357                .null_bit_buffer(null_buffer);
358            self.create_array_from_builder(builder)
359        } else {
360            unreachable!("Cannot create dictionary array from {:?}", data_type)
361        }
362    }
363}
364
365/// State for decoding Arrow arrays from an [IPC RecordBatch] structure to
366/// [`RecordBatch`]
367///
368/// [IPC RecordBatch]: crate::RecordBatch
369struct RecordBatchDecoder<'a> {
370    /// The flatbuffers encoded record batch
371    batch: crate::RecordBatch<'a>,
372    /// The output schema
373    schema: SchemaRef,
374    /// Decoded dictionaries indexed by dictionary id
375    dictionaries_by_id: &'a HashMap<i64, ArrayRef>,
376    /// Optional compression codec
377    compression: Option<CompressionCodec>,
378    /// The format version
379    version: MetadataVersion,
380    /// The raw data buffer
381    data: &'a Buffer,
382    /// The fields comprising this array
383    nodes: VectorIter<'a, FieldNode>,
384    /// The buffers comprising this array
385    buffers: VectorIter<'a, crate::Buffer>,
386    /// Projection (subset of columns) to read, if any
387    /// See [`RecordBatchDecoder::with_projection`] for details
388    projection: Option<&'a [usize]>,
389    /// Are buffers required to already be aligned? See
390    /// [`RecordBatchDecoder::with_require_alignment`] for details
391    require_alignment: bool,
392    /// Should validation be skipped when reading data? Defaults to false.
393    ///
394    /// See [`FileDecoder::with_skip_validation`] for details.
395    skip_validation: UnsafeFlag,
396}
397
398impl<'a> RecordBatchDecoder<'a> {
399    /// Create a reader for decoding arrays from an encoded [`RecordBatch`]
400    fn try_new(
401        buf: &'a Buffer,
402        batch: crate::RecordBatch<'a>,
403        schema: SchemaRef,
404        dictionaries_by_id: &'a HashMap<i64, ArrayRef>,
405        metadata: &'a MetadataVersion,
406    ) -> Result<Self, ArrowError> {
407        let buffers = batch.buffers().ok_or_else(|| {
408            ArrowError::IpcError("Unable to get buffers from IPC RecordBatch".to_string())
409        })?;
410        let field_nodes = batch.nodes().ok_or_else(|| {
411            ArrowError::IpcError("Unable to get field nodes from IPC RecordBatch".to_string())
412        })?;
413
414        let batch_compression = batch.compression();
415        let compression = batch_compression
416            .map(|batch_compression| batch_compression.codec().try_into())
417            .transpose()?;
418
419        Ok(Self {
420            batch,
421            schema,
422            dictionaries_by_id,
423            compression,
424            version: *metadata,
425            data: buf,
426            nodes: field_nodes.iter(),
427            buffers: buffers.iter(),
428            projection: None,
429            require_alignment: false,
430            skip_validation: UnsafeFlag::new(),
431        })
432    }
433
434    /// Set the projection (default: None)
435    ///
436    /// If set, the projection is the list  of column indices
437    /// that will be read
438    pub fn with_projection(mut self, projection: Option<&'a [usize]>) -> Self {
439        self.projection = projection;
440        self
441    }
442
443    /// Set require_alignment (default: false)
444    ///
445    /// If true, buffers must be aligned appropriately or error will
446    /// result. If false, buffers will be copied to aligned buffers
447    /// if necessary.
448    pub fn with_require_alignment(mut self, require_alignment: bool) -> Self {
449        self.require_alignment = require_alignment;
450        self
451    }
452
453    /// Specifies if validation should be skipped when reading data (defaults to `false`)
454    ///
455    /// Note this API is somewhat "funky" as it allows the caller to skip validation
456    /// without having to use `unsafe` code. If this is ever made public
457    /// it should be made clearer that this is a potentially unsafe by
458    /// using an `unsafe` function that takes a boolean flag.
459    ///
460    /// # Safety
461    ///
462    /// Relies on the caller only passing a flag with `true` value if they are
463    /// certain that the data is valid
464    pub(crate) fn with_skip_validation(mut self, skip_validation: UnsafeFlag) -> Self {
465        self.skip_validation = skip_validation;
466        self
467    }
468
469    /// Read the record batch, consuming the reader
470    fn read_record_batch(mut self) -> Result<RecordBatch, ArrowError> {
471        let mut variadic_counts: VecDeque<i64> = self
472            .batch
473            .variadicBufferCounts()
474            .into_iter()
475            .flatten()
476            .collect();
477
478        let options = RecordBatchOptions::new().with_row_count(Some(self.batch.length() as usize));
479
480        let schema = Arc::clone(&self.schema);
481        if let Some(projection) = self.projection {
482            let mut arrays = vec![];
483            // project fields
484            for (idx, field) in schema.fields().iter().enumerate() {
485                // Create array for projected field
486                if let Some(proj_idx) = projection.iter().position(|p| p == &idx) {
487                    let child = self.create_array(field, &mut variadic_counts)?;
488                    arrays.push((proj_idx, child));
489                } else {
490                    self.skip_field(field, &mut variadic_counts)?;
491                }
492            }
493
494            arrays.sort_by_key(|t| t.0);
495
496            let schema = Arc::new(schema.project(projection)?);
497            let columns = arrays.into_iter().map(|t| t.1).collect::<Vec<_>>();
498
499            if self.skip_validation.get() {
500                // Safety: setting `skip_validation` requires `unsafe`, user assures data is valid
501                unsafe {
502                    Ok(RecordBatch::new_unchecked(
503                        schema,
504                        columns,
505                        self.batch.length() as usize,
506                    ))
507                }
508            } else {
509                assert!(variadic_counts.is_empty());
510                RecordBatch::try_new_with_options(schema, columns, &options)
511            }
512        } else {
513            let mut children = vec![];
514            // keep track of index as lists require more than one node
515            for field in schema.fields() {
516                let child = self.create_array(field, &mut variadic_counts)?;
517                children.push(child);
518            }
519
520            if self.skip_validation.get() {
521                // Safety: setting `skip_validation` requires `unsafe`, user assures data is valid
522                unsafe {
523                    Ok(RecordBatch::new_unchecked(
524                        schema,
525                        children,
526                        self.batch.length() as usize,
527                    ))
528                }
529            } else {
530                assert!(variadic_counts.is_empty());
531                RecordBatch::try_new_with_options(schema, children, &options)
532            }
533        }
534    }
535
536    fn next_buffer(&mut self) -> Result<Buffer, ArrowError> {
537        read_buffer(self.buffers.next().unwrap(), self.data, self.compression)
538    }
539
540    fn skip_buffer(&mut self) {
541        self.buffers.next().unwrap();
542    }
543
544    fn next_node(&mut self, field: &Field) -> Result<&'a FieldNode, ArrowError> {
545        self.nodes.next().ok_or_else(|| {
546            ArrowError::SchemaError(format!(
547                "Invalid data for schema. {field} refers to node not found in schema",
548            ))
549        })
550    }
551
552    fn skip_field(
553        &mut self,
554        field: &Field,
555        variadic_count: &mut VecDeque<i64>,
556    ) -> Result<(), ArrowError> {
557        self.next_node(field)?;
558
559        match field.data_type() {
560            Utf8 | Binary | LargeBinary | LargeUtf8 => {
561                for _ in 0..3 {
562                    self.skip_buffer()
563                }
564            }
565            Utf8View | BinaryView => {
566                let count = variadic_count
567                    .pop_front()
568                    .ok_or(ArrowError::IpcError(format!(
569                        "Missing variadic count for {} column",
570                        field.data_type()
571                    )))?;
572                let count = count + 2; // view and null buffer.
573                for _i in 0..count {
574                    self.skip_buffer()
575                }
576            }
577            FixedSizeBinary(_) => {
578                self.skip_buffer();
579                self.skip_buffer();
580            }
581            List(list_field) | LargeList(list_field) | Map(list_field, _) => {
582                self.skip_buffer();
583                self.skip_buffer();
584                self.skip_field(list_field, variadic_count)?;
585            }
586            FixedSizeList(list_field, _) => {
587                self.skip_buffer();
588                self.skip_field(list_field, variadic_count)?;
589            }
590            Struct(struct_fields) => {
591                self.skip_buffer();
592
593                // skip for each field
594                for struct_field in struct_fields {
595                    self.skip_field(struct_field, variadic_count)?
596                }
597            }
598            RunEndEncoded(run_ends_field, values_field) => {
599                self.skip_field(run_ends_field, variadic_count)?;
600                self.skip_field(values_field, variadic_count)?;
601            }
602            Dictionary(_, _) => {
603                self.skip_buffer(); // Nulls
604                self.skip_buffer(); // Indices
605            }
606            Union(fields, mode) => {
607                self.skip_buffer(); // Nulls
608
609                match mode {
610                    UnionMode::Dense => self.skip_buffer(),
611                    UnionMode::Sparse => {}
612                };
613
614                for (_, field) in fields.iter() {
615                    self.skip_field(field, variadic_count)?
616                }
617            }
618            Null => {} // No buffer increases
619            _ => {
620                self.skip_buffer();
621                self.skip_buffer();
622            }
623        };
624        Ok(())
625    }
626}
627
628/// Creates a record batch from binary data using the `crate::RecordBatch` indexes and the `Schema`.
629///
630/// If `require_alignment` is true, this function will return an error if any array data in the
631/// input `buf` is not properly aligned.
632/// Under the hood it will use [`arrow_data::ArrayDataBuilder::build`] to construct [`arrow_data::ArrayData`].
633///
634/// If `require_alignment` is false, this function will automatically allocate a new aligned buffer
635/// and copy over the data if any array data in the input `buf` is not properly aligned.
636/// (Properly aligned array data will remain zero-copy.)
637/// Under the hood it will use [`arrow_data::ArrayDataBuilder::build_aligned`] to construct [`arrow_data::ArrayData`].
638pub fn read_record_batch(
639    buf: &Buffer,
640    batch: crate::RecordBatch,
641    schema: SchemaRef,
642    dictionaries_by_id: &HashMap<i64, ArrayRef>,
643    projection: Option<&[usize]>,
644    metadata: &MetadataVersion,
645) -> Result<RecordBatch, ArrowError> {
646    RecordBatchDecoder::try_new(buf, batch, schema, dictionaries_by_id, metadata)?
647        .with_projection(projection)
648        .with_require_alignment(false)
649        .read_record_batch()
650}
651
652/// Read the dictionary from the buffer and provided metadata,
653/// updating the `dictionaries_by_id` with the resulting dictionary
654pub fn read_dictionary(
655    buf: &Buffer,
656    batch: crate::DictionaryBatch,
657    schema: &Schema,
658    dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
659    metadata: &MetadataVersion,
660) -> Result<(), ArrowError> {
661    read_dictionary_impl(
662        buf,
663        batch,
664        schema,
665        dictionaries_by_id,
666        metadata,
667        false,
668        UnsafeFlag::new(),
669    )
670}
671
672fn read_dictionary_impl(
673    buf: &Buffer,
674    batch: crate::DictionaryBatch,
675    schema: &Schema,
676    dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
677    metadata: &MetadataVersion,
678    require_alignment: bool,
679    skip_validation: UnsafeFlag,
680) -> Result<(), ArrowError> {
681    if batch.isDelta() {
682        return Err(ArrowError::InvalidArgumentError(
683            "delta dictionary batches not supported".to_string(),
684        ));
685    }
686
687    let id = batch.id();
688    #[allow(deprecated)]
689    let fields_using_this_dictionary = schema.fields_with_dict_id(id);
690    let first_field = fields_using_this_dictionary.first().ok_or_else(|| {
691        ArrowError::InvalidArgumentError(format!("dictionary id {id} not found in schema"))
692    })?;
693
694    // As the dictionary batch does not contain the type of the
695    // values array, we need to retrieve this from the schema.
696    // Get an array representing this dictionary's values.
697    let dictionary_values: ArrayRef = match first_field.data_type() {
698        DataType::Dictionary(_, ref value_type) => {
699            // Make a fake schema for the dictionary batch.
700            let value = value_type.as_ref().clone();
701            let schema = Schema::new(vec![Field::new("", value, true)]);
702            // Read a single column
703            let record_batch = RecordBatchDecoder::try_new(
704                buf,
705                batch.data().unwrap(),
706                Arc::new(schema),
707                dictionaries_by_id,
708                metadata,
709            )?
710            .with_require_alignment(require_alignment)
711            .with_skip_validation(skip_validation)
712            .read_record_batch()?;
713
714            Some(record_batch.column(0).clone())
715        }
716        _ => None,
717    }
718    .ok_or_else(|| {
719        ArrowError::InvalidArgumentError(format!("dictionary id {id} not found in schema"))
720    })?;
721
722    // We don't currently record the isOrdered field. This could be general
723    // attributes of arrays.
724    // Add (possibly multiple) array refs to the dictionaries array.
725    dictionaries_by_id.insert(id, dictionary_values.clone());
726
727    Ok(())
728}
729
730/// Read the data for a given block
731fn read_block<R: Read + Seek>(mut reader: R, block: &Block) -> Result<Buffer, ArrowError> {
732    reader.seek(SeekFrom::Start(block.offset() as u64))?;
733    let body_len = block.bodyLength().to_usize().unwrap();
734    let metadata_len = block.metaDataLength().to_usize().unwrap();
735    let total_len = body_len.checked_add(metadata_len).unwrap();
736
737    let mut buf = MutableBuffer::from_len_zeroed(total_len);
738    reader.read_exact(&mut buf)?;
739    Ok(buf.into())
740}
741
742/// Parse an encapsulated message
743///
744/// <https://arrow.apache.org/docs/format/Columnar.html#encapsulated-message-format>
745fn parse_message(buf: &[u8]) -> Result<Message, ArrowError> {
746    let buf = match buf[..4] == CONTINUATION_MARKER {
747        true => &buf[8..],
748        false => &buf[4..],
749    };
750    crate::root_as_message(buf)
751        .map_err(|err| ArrowError::ParseError(format!("Unable to get root as message: {err:?}")))
752}
753
754/// Read the footer length from the last 10 bytes of an Arrow IPC file
755///
756/// Expects a 4 byte footer length followed by `b"ARROW1"`
757pub fn read_footer_length(buf: [u8; 10]) -> Result<usize, ArrowError> {
758    if buf[4..] != super::ARROW_MAGIC {
759        return Err(ArrowError::ParseError(
760            "Arrow file does not contain correct footer".to_string(),
761        ));
762    }
763
764    // read footer length
765    let footer_len = i32::from_le_bytes(buf[..4].try_into().unwrap());
766    footer_len
767        .try_into()
768        .map_err(|_| ArrowError::ParseError(format!("Invalid footer length: {footer_len}")))
769}
770
771/// A low-level, push-based interface for reading an IPC file
772///
773/// For a higher-level interface see [`FileReader`]
774///
775/// For an example of using this API with `mmap` see the [`zero_copy_ipc`] example.
776///
777/// [`zero_copy_ipc`]: https://github.com/apache/arrow-rs/blob/main/arrow/examples/zero_copy_ipc.rs
778///
779/// ```
780/// # use std::sync::Arc;
781/// # use arrow_array::*;
782/// # use arrow_array::types::Int32Type;
783/// # use arrow_buffer::Buffer;
784/// # use arrow_ipc::convert::fb_to_schema;
785/// # use arrow_ipc::reader::{FileDecoder, read_footer_length};
786/// # use arrow_ipc::root_as_footer;
787/// # use arrow_ipc::writer::FileWriter;
788/// // Write an IPC file
789///
790/// let batch = RecordBatch::try_from_iter([
791///     ("a", Arc::new(Int32Array::from(vec![1, 2, 3])) as _),
792///     ("b", Arc::new(Int32Array::from(vec![1, 2, 3])) as _),
793///     ("c", Arc::new(DictionaryArray::<Int32Type>::from_iter(["hello", "hello", "world"])) as _),
794/// ]).unwrap();
795///
796/// let schema = batch.schema();
797///
798/// let mut out = Vec::with_capacity(1024);
799/// let mut writer = FileWriter::try_new(&mut out, schema.as_ref()).unwrap();
800/// writer.write(&batch).unwrap();
801/// writer.finish().unwrap();
802///
803/// drop(writer);
804///
805/// // Read IPC file
806///
807/// let buffer = Buffer::from_vec(out);
808/// let trailer_start = buffer.len() - 10;
809/// let footer_len = read_footer_length(buffer[trailer_start..].try_into().unwrap()).unwrap();
810/// let footer = root_as_footer(&buffer[trailer_start - footer_len..trailer_start]).unwrap();
811///
812/// let back = fb_to_schema(footer.schema().unwrap());
813/// assert_eq!(&back, schema.as_ref());
814///
815/// let mut decoder = FileDecoder::new(schema, footer.version());
816///
817/// // Read dictionaries
818/// for block in footer.dictionaries().iter().flatten() {
819///     let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
820///     let data = buffer.slice_with_length(block.offset() as _, block_len);
821///     decoder.read_dictionary(&block, &data).unwrap();
822/// }
823///
824/// // Read record batch
825/// let batches = footer.recordBatches().unwrap();
826/// assert_eq!(batches.len(), 1); // Only wrote a single batch
827///
828/// let block = batches.get(0);
829/// let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
830/// let data = buffer.slice_with_length(block.offset() as _, block_len);
831/// let back = decoder.read_record_batch(block, &data).unwrap().unwrap();
832///
833/// assert_eq!(batch, back);
834/// ```
835#[derive(Debug)]
836pub struct FileDecoder {
837    schema: SchemaRef,
838    dictionaries: HashMap<i64, ArrayRef>,
839    version: MetadataVersion,
840    projection: Option<Vec<usize>>,
841    require_alignment: bool,
842    skip_validation: UnsafeFlag,
843}
844
845impl FileDecoder {
846    /// Create a new [`FileDecoder`] with the given schema and version
847    pub fn new(schema: SchemaRef, version: MetadataVersion) -> Self {
848        Self {
849            schema,
850            version,
851            dictionaries: Default::default(),
852            projection: None,
853            require_alignment: false,
854            skip_validation: UnsafeFlag::new(),
855        }
856    }
857
858    /// Specify a projection
859    pub fn with_projection(mut self, projection: Vec<usize>) -> Self {
860        self.projection = Some(projection);
861        self
862    }
863
864    /// Specifies if the array data in input buffers is required to be properly aligned.
865    ///
866    /// If `require_alignment` is true, this decoder will return an error if any array data in the
867    /// input `buf` is not properly aligned.
868    /// Under the hood it will use [`arrow_data::ArrayDataBuilder::build`] to construct
869    /// [`arrow_data::ArrayData`].
870    ///
871    /// If `require_alignment` is false (the default), this decoder will automatically allocate a
872    /// new aligned buffer and copy over the data if any array data in the input `buf` is not
873    /// properly aligned. (Properly aligned array data will remain zero-copy.)
874    /// Under the hood it will use [`arrow_data::ArrayDataBuilder::build_aligned`] to construct
875    /// [`arrow_data::ArrayData`].
876    pub fn with_require_alignment(mut self, require_alignment: bool) -> Self {
877        self.require_alignment = require_alignment;
878        self
879    }
880
881    /// Specifies if validation should be skipped when reading data (defaults to `false`)
882    ///
883    /// # Safety
884    ///
885    /// This flag must only be set to `true` when you trust the input data and are sure the data you are
886    /// reading is a valid Arrow IPC file, otherwise undefined behavior may
887    /// result.
888    ///
889    /// For example, some programs may wish to trust reading IPC files written
890    /// by the same process that created the files.
891    pub unsafe fn with_skip_validation(mut self, skip_validation: bool) -> Self {
892        self.skip_validation.set(skip_validation);
893        self
894    }
895
896    fn read_message<'a>(&self, buf: &'a [u8]) -> Result<Message<'a>, ArrowError> {
897        let message = parse_message(buf)?;
898
899        // some old test data's footer metadata is not set, so we account for that
900        if self.version != MetadataVersion::V1 && message.version() != self.version {
901            return Err(ArrowError::IpcError(
902                "Could not read IPC message as metadata versions mismatch".to_string(),
903            ));
904        }
905        Ok(message)
906    }
907
908    /// Read the dictionary with the given block and data buffer
909    pub fn read_dictionary(&mut self, block: &Block, buf: &Buffer) -> Result<(), ArrowError> {
910        let message = self.read_message(buf)?;
911        match message.header_type() {
912            crate::MessageHeader::DictionaryBatch => {
913                let batch = message.header_as_dictionary_batch().unwrap();
914                read_dictionary_impl(
915                    &buf.slice(block.metaDataLength() as _),
916                    batch,
917                    &self.schema,
918                    &mut self.dictionaries,
919                    &message.version(),
920                    self.require_alignment,
921                    self.skip_validation.clone(),
922                )
923            }
924            t => Err(ArrowError::ParseError(format!(
925                "Expecting DictionaryBatch in dictionary blocks, found {t:?}."
926            ))),
927        }
928    }
929
930    /// Read the RecordBatch with the given block and data buffer
931    pub fn read_record_batch(
932        &self,
933        block: &Block,
934        buf: &Buffer,
935    ) -> Result<Option<RecordBatch>, ArrowError> {
936        let message = self.read_message(buf)?;
937        match message.header_type() {
938            crate::MessageHeader::Schema => Err(ArrowError::IpcError(
939                "Not expecting a schema when messages are read".to_string(),
940            )),
941            crate::MessageHeader::RecordBatch => {
942                let batch = message.header_as_record_batch().ok_or_else(|| {
943                    ArrowError::IpcError("Unable to read IPC message as record batch".to_string())
944                })?;
945                // read the block that makes up the record batch into a buffer
946                RecordBatchDecoder::try_new(
947                    &buf.slice(block.metaDataLength() as _),
948                    batch,
949                    self.schema.clone(),
950                    &self.dictionaries,
951                    &message.version(),
952                )?
953                .with_projection(self.projection.as_deref())
954                .with_require_alignment(self.require_alignment)
955                .with_skip_validation(self.skip_validation.clone())
956                .read_record_batch()
957                .map(Some)
958            }
959            crate::MessageHeader::NONE => Ok(None),
960            t => Err(ArrowError::InvalidArgumentError(format!(
961                "Reading types other than record batches not yet supported, unable to read {t:?}"
962            ))),
963        }
964    }
965}
966
967/// Build an Arrow [`FileReader`] with custom options.
968#[derive(Debug)]
969pub struct FileReaderBuilder {
970    /// Optional projection for which columns to load (zero-based column indices)
971    projection: Option<Vec<usize>>,
972    /// Passed through to construct [`VerifierOptions`]
973    max_footer_fb_tables: usize,
974    /// Passed through to construct [`VerifierOptions`]
975    max_footer_fb_depth: usize,
976}
977
978impl Default for FileReaderBuilder {
979    fn default() -> Self {
980        let verifier_options = VerifierOptions::default();
981        Self {
982            max_footer_fb_tables: verifier_options.max_tables,
983            max_footer_fb_depth: verifier_options.max_depth,
984            projection: None,
985        }
986    }
987}
988
989impl FileReaderBuilder {
990    /// Options for creating a new [`FileReader`].
991    ///
992    /// To convert a builder into a reader, call [`FileReaderBuilder::build`].
993    pub fn new() -> Self {
994        Self::default()
995    }
996
997    /// Optional projection for which columns to load (zero-based column indices).
998    pub fn with_projection(mut self, projection: Vec<usize>) -> Self {
999        self.projection = Some(projection);
1000        self
1001    }
1002
1003    /// Flatbuffers option for parsing the footer. Controls the max number of fields and
1004    /// metadata key-value pairs that can be parsed from the schema of the footer.
1005    ///
1006    /// By default this is set to `1_000_000` which roughly translates to a schema with
1007    /// no metadata key-value pairs but 499,999 fields.
1008    ///
1009    /// This default limit is enforced to protect against malicious files with a massive
1010    /// amount of flatbuffer tables which could cause a denial of service attack.
1011    ///
1012    /// If you need to ingest a trusted file with a massive number of fields and/or
1013    /// metadata key-value pairs and are facing the error `"Unable to get root as
1014    /// footer: TooManyTables"` then increase this parameter as necessary.
1015    pub fn with_max_footer_fb_tables(mut self, max_footer_fb_tables: usize) -> Self {
1016        self.max_footer_fb_tables = max_footer_fb_tables;
1017        self
1018    }
1019
1020    /// Flatbuffers option for parsing the footer. Controls the max depth for schemas with
1021    /// nested fields parsed from the footer.
1022    ///
1023    /// By default this is set to `64` which roughly translates to a schema with
1024    /// a field nested 60 levels down through other struct fields.
1025    ///
1026    /// This default limit is enforced to protect against malicious files with a extremely
1027    /// deep flatbuffer structure which could cause a denial of service attack.
1028    ///
1029    /// If you need to ingest a trusted file with a deeply nested field and are facing the
1030    /// error `"Unable to get root as footer: DepthLimitReached"` then increase this
1031    /// parameter as necessary.
1032    pub fn with_max_footer_fb_depth(mut self, max_footer_fb_depth: usize) -> Self {
1033        self.max_footer_fb_depth = max_footer_fb_depth;
1034        self
1035    }
1036
1037    /// Build [`FileReader`] with given reader.
1038    pub fn build<R: Read + Seek>(self, mut reader: R) -> Result<FileReader<R>, ArrowError> {
1039        // Space for ARROW_MAGIC (6 bytes) and length (4 bytes)
1040        let mut buffer = [0; 10];
1041        reader.seek(SeekFrom::End(-10))?;
1042        reader.read_exact(&mut buffer)?;
1043
1044        let footer_len = read_footer_length(buffer)?;
1045
1046        // read footer
1047        let mut footer_data = vec![0; footer_len];
1048        reader.seek(SeekFrom::End(-10 - footer_len as i64))?;
1049        reader.read_exact(&mut footer_data)?;
1050
1051        let verifier_options = VerifierOptions {
1052            max_tables: self.max_footer_fb_tables,
1053            max_depth: self.max_footer_fb_depth,
1054            ..Default::default()
1055        };
1056        let footer = crate::root_as_footer_with_opts(&verifier_options, &footer_data[..]).map_err(
1057            |err| ArrowError::ParseError(format!("Unable to get root as footer: {err:?}")),
1058        )?;
1059
1060        let blocks = footer.recordBatches().ok_or_else(|| {
1061            ArrowError::ParseError("Unable to get record batches from IPC Footer".to_string())
1062        })?;
1063
1064        let total_blocks = blocks.len();
1065
1066        let ipc_schema = footer.schema().unwrap();
1067        if !ipc_schema.endianness().equals_to_target_endianness() {
1068            return Err(ArrowError::IpcError(
1069                "the endianness of the source system does not match the endianness of the target system.".to_owned()
1070            ));
1071        }
1072
1073        let schema = crate::convert::fb_to_schema(ipc_schema);
1074
1075        let mut custom_metadata = HashMap::new();
1076        if let Some(fb_custom_metadata) = footer.custom_metadata() {
1077            for kv in fb_custom_metadata.into_iter() {
1078                custom_metadata.insert(
1079                    kv.key().unwrap().to_string(),
1080                    kv.value().unwrap().to_string(),
1081                );
1082            }
1083        }
1084
1085        let mut decoder = FileDecoder::new(Arc::new(schema), footer.version());
1086        if let Some(projection) = self.projection {
1087            decoder = decoder.with_projection(projection)
1088        }
1089
1090        // Create an array of optional dictionary value arrays, one per field.
1091        if let Some(dictionaries) = footer.dictionaries() {
1092            for block in dictionaries {
1093                let buf = read_block(&mut reader, block)?;
1094                decoder.read_dictionary(block, &buf)?;
1095            }
1096        }
1097
1098        Ok(FileReader {
1099            reader,
1100            blocks: blocks.iter().copied().collect(),
1101            current_block: 0,
1102            total_blocks,
1103            decoder,
1104            custom_metadata,
1105        })
1106    }
1107}
1108
1109/// Arrow File Reader
1110///
1111/// Reads Arrow [`RecordBatch`]es from bytes in the [IPC File Format],
1112/// providing random access to the record batches.
1113///
1114/// # See Also
1115///
1116/// * [`Self::set_index`] for random access
1117/// * [`StreamReader`] for reading streaming data
1118///
1119/// # Example: Reading from a `File`
1120/// ```
1121/// # use std::io::Cursor;
1122/// use arrow_array::record_batch;
1123/// # use arrow_ipc::reader::FileReader;
1124/// # use arrow_ipc::writer::FileWriter;
1125/// # let batch = record_batch!(("a", Int32, [1, 2, 3])).unwrap();
1126/// # let mut file = vec![]; // mimic a stream for the example
1127/// # {
1128/// #  let mut writer = FileWriter::try_new(&mut file, &batch.schema()).unwrap();
1129/// #  writer.write(&batch).unwrap();
1130/// #  writer.write(&batch).unwrap();
1131/// #  writer.finish().unwrap();
1132/// # }
1133/// # let mut file = Cursor::new(&file);
1134/// let projection = None; // read all columns
1135/// let mut reader = FileReader::try_new(&mut file, projection).unwrap();
1136/// // Position the reader to the second batch
1137/// reader.set_index(1).unwrap();
1138/// // read batches from the reader using the Iterator trait
1139/// let mut num_rows = 0;
1140/// for batch in reader {
1141///    let batch = batch.unwrap();
1142///    num_rows += batch.num_rows();
1143/// }
1144/// assert_eq!(num_rows, 3);
1145/// ```
1146/// # Example: Reading from `mmap`ed file
1147///
1148/// For an example creating Arrays without copying using  memory mapped (`mmap`)
1149/// files see the [`zero_copy_ipc`] example.
1150///
1151/// [IPC File Format]: https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format
1152/// [`zero_copy_ipc`]: https://github.com/apache/arrow-rs/blob/main/arrow/examples/zero_copy_ipc.rs
1153pub struct FileReader<R> {
1154    /// File reader that supports reading and seeking
1155    reader: R,
1156
1157    /// The decoder
1158    decoder: FileDecoder,
1159
1160    /// The blocks in the file
1161    ///
1162    /// A block indicates the regions in the file to read to get data
1163    blocks: Vec<Block>,
1164
1165    /// A counter to keep track of the current block that should be read
1166    current_block: usize,
1167
1168    /// The total number of blocks, which may contain record batches and other types
1169    total_blocks: usize,
1170
1171    /// User defined metadata
1172    custom_metadata: HashMap<String, String>,
1173}
1174
1175impl<R> fmt::Debug for FileReader<R> {
1176    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> {
1177        f.debug_struct("FileReader<R>")
1178            .field("decoder", &self.decoder)
1179            .field("blocks", &self.blocks)
1180            .field("current_block", &self.current_block)
1181            .field("total_blocks", &self.total_blocks)
1182            .finish_non_exhaustive()
1183    }
1184}
1185
1186impl<R: Read + Seek> FileReader<BufReader<R>> {
1187    /// Try to create a new file reader with the reader wrapped in a BufReader.
1188    ///
1189    /// See [`FileReader::try_new`] for an unbuffered version.
1190    pub fn try_new_buffered(reader: R, projection: Option<Vec<usize>>) -> Result<Self, ArrowError> {
1191        Self::try_new(BufReader::new(reader), projection)
1192    }
1193}
1194
1195impl<R: Read + Seek> FileReader<R> {
1196    /// Try to create a new file reader.
1197    ///
1198    /// There is no internal buffering. If buffered reads are needed you likely want to use
1199    /// [`FileReader::try_new_buffered`] instead.    
1200    ///
1201    /// # Errors
1202    ///
1203    /// An ['Err'](Result::Err) may be returned if:
1204    /// - the file does not meet the Arrow Format footer requirements, or
1205    /// - file endianness does not match the target endianness.
1206    pub fn try_new(reader: R, projection: Option<Vec<usize>>) -> Result<Self, ArrowError> {
1207        let builder = FileReaderBuilder {
1208            projection,
1209            ..Default::default()
1210        };
1211        builder.build(reader)
1212    }
1213
1214    /// Return user defined customized metadata
1215    pub fn custom_metadata(&self) -> &HashMap<String, String> {
1216        &self.custom_metadata
1217    }
1218
1219    /// Return the number of batches in the file
1220    pub fn num_batches(&self) -> usize {
1221        self.total_blocks
1222    }
1223
1224    /// Return the schema of the file
1225    pub fn schema(&self) -> SchemaRef {
1226        self.decoder.schema.clone()
1227    }
1228
1229    /// See to a specific [`RecordBatch`]
1230    ///
1231    /// Sets the current block to the index, allowing random reads
1232    pub fn set_index(&mut self, index: usize) -> Result<(), ArrowError> {
1233        if index >= self.total_blocks {
1234            Err(ArrowError::InvalidArgumentError(format!(
1235                "Cannot set batch to index {} from {} total batches",
1236                index, self.total_blocks
1237            )))
1238        } else {
1239            self.current_block = index;
1240            Ok(())
1241        }
1242    }
1243
1244    fn maybe_next(&mut self) -> Result<Option<RecordBatch>, ArrowError> {
1245        let block = &self.blocks[self.current_block];
1246        self.current_block += 1;
1247
1248        // read length
1249        let buffer = read_block(&mut self.reader, block)?;
1250        self.decoder.read_record_batch(block, &buffer)
1251    }
1252
1253    /// Gets a reference to the underlying reader.
1254    ///
1255    /// It is inadvisable to directly read from the underlying reader.
1256    pub fn get_ref(&self) -> &R {
1257        &self.reader
1258    }
1259
1260    /// Gets a mutable reference to the underlying reader.
1261    ///
1262    /// It is inadvisable to directly read from the underlying reader.
1263    pub fn get_mut(&mut self) -> &mut R {
1264        &mut self.reader
1265    }
1266
1267    /// Specifies if validation should be skipped when reading data (defaults to `false`)
1268    ///
1269    /// # Safety
1270    ///
1271    /// See [`FileDecoder::with_skip_validation`]
1272    pub unsafe fn with_skip_validation(mut self, skip_validation: bool) -> Self {
1273        self.decoder = self.decoder.with_skip_validation(skip_validation);
1274        self
1275    }
1276}
1277
1278impl<R: Read + Seek> Iterator for FileReader<R> {
1279    type Item = Result<RecordBatch, ArrowError>;
1280
1281    fn next(&mut self) -> Option<Self::Item> {
1282        // get current block
1283        if self.current_block < self.total_blocks {
1284            self.maybe_next().transpose()
1285        } else {
1286            None
1287        }
1288    }
1289}
1290
1291impl<R: Read + Seek> RecordBatchReader for FileReader<R> {
1292    fn schema(&self) -> SchemaRef {
1293        self.schema()
1294    }
1295}
1296
1297/// Arrow Stream Reader
1298///
1299/// Reads Arrow [`RecordBatch`]es from bytes in the [IPC Streaming Format].
1300///
1301/// # See Also
1302///
1303/// * [`FileReader`] for random access.
1304///
1305/// # Example
1306/// ```
1307/// # use arrow_array::record_batch;
1308/// # use arrow_ipc::reader::StreamReader;
1309/// # use arrow_ipc::writer::StreamWriter;
1310/// # let batch = record_batch!(("a", Int32, [1, 2, 3])).unwrap();
1311/// # let mut stream = vec![]; // mimic a stream for the example
1312/// # {
1313/// #  let mut writer = StreamWriter::try_new(&mut stream, &batch.schema()).unwrap();
1314/// #  writer.write(&batch).unwrap();
1315/// #  writer.finish().unwrap();
1316/// # }
1317/// # let stream = stream.as_slice();
1318/// let projection = None; // read all columns
1319/// let mut reader = StreamReader::try_new(stream, projection).unwrap();
1320/// // read batches from the reader using the Iterator trait
1321/// let mut num_rows = 0;
1322/// for batch in reader {
1323///    let batch = batch.unwrap();
1324///    num_rows += batch.num_rows();
1325/// }
1326/// assert_eq!(num_rows, 3);
1327/// ```
1328///
1329/// [IPC Streaming Format]: https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format
1330pub struct StreamReader<R> {
1331    /// Stream reader
1332    reader: R,
1333
1334    /// The schema that is read from the stream's first message
1335    schema: SchemaRef,
1336
1337    /// Optional dictionaries for each schema field.
1338    ///
1339    /// Dictionaries may be appended to in the streaming format.
1340    dictionaries_by_id: HashMap<i64, ArrayRef>,
1341
1342    /// An indicator of whether the stream is complete.
1343    ///
1344    /// This value is set to `true` the first time the reader's `next()` returns `None`.
1345    finished: bool,
1346
1347    /// Optional projection
1348    projection: Option<(Vec<usize>, Schema)>,
1349
1350    /// Should validation be skipped when reading data? Defaults to false.
1351    ///
1352    /// See [`FileDecoder::with_skip_validation`] for details.
1353    skip_validation: UnsafeFlag,
1354}
1355
1356impl<R> fmt::Debug for StreamReader<R> {
1357    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> std::result::Result<(), fmt::Error> {
1358        f.debug_struct("StreamReader<R>")
1359            .field("reader", &"R")
1360            .field("schema", &self.schema)
1361            .field("dictionaries_by_id", &self.dictionaries_by_id)
1362            .field("finished", &self.finished)
1363            .field("projection", &self.projection)
1364            .finish()
1365    }
1366}
1367
1368impl<R: Read> StreamReader<BufReader<R>> {
1369    /// Try to create a new stream reader with the reader wrapped in a BufReader.
1370    ///
1371    /// See [`StreamReader::try_new`] for an unbuffered version.
1372    pub fn try_new_buffered(reader: R, projection: Option<Vec<usize>>) -> Result<Self, ArrowError> {
1373        Self::try_new(BufReader::new(reader), projection)
1374    }
1375}
1376
1377impl<R: Read> StreamReader<R> {
1378    /// Try to create a new stream reader.
1379    ///
1380    /// To check if the reader is done, use [`is_finished(self)`](StreamReader::is_finished).
1381    ///
1382    /// There is no internal buffering. If buffered reads are needed you likely want to use
1383    /// [`StreamReader::try_new_buffered`] instead.
1384    ///
1385    /// # Errors
1386    ///
1387    /// An ['Err'](Result::Err) may be returned if the reader does not encounter a schema
1388    /// as the first message in the stream.
1389    pub fn try_new(
1390        mut reader: R,
1391        projection: Option<Vec<usize>>,
1392    ) -> Result<StreamReader<R>, ArrowError> {
1393        // determine metadata length
1394        let mut meta_size: [u8; 4] = [0; 4];
1395        reader.read_exact(&mut meta_size)?;
1396        let meta_len = {
1397            // If a continuation marker is encountered, skip over it and read
1398            // the size from the next four bytes.
1399            if meta_size == CONTINUATION_MARKER {
1400                reader.read_exact(&mut meta_size)?;
1401            }
1402            i32::from_le_bytes(meta_size)
1403        };
1404
1405        let mut meta_buffer = vec![0; meta_len as usize];
1406        reader.read_exact(&mut meta_buffer)?;
1407
1408        let message = crate::root_as_message(meta_buffer.as_slice()).map_err(|err| {
1409            ArrowError::ParseError(format!("Unable to get root as message: {err:?}"))
1410        })?;
1411        // message header is a Schema, so read it
1412        let ipc_schema: crate::Schema = message.header_as_schema().ok_or_else(|| {
1413            ArrowError::ParseError("Unable to read IPC message as schema".to_string())
1414        })?;
1415        let schema = crate::convert::fb_to_schema(ipc_schema);
1416
1417        // Create an array of optional dictionary value arrays, one per field.
1418        let dictionaries_by_id = HashMap::new();
1419
1420        let projection = match projection {
1421            Some(projection_indices) => {
1422                let schema = schema.project(&projection_indices)?;
1423                Some((projection_indices, schema))
1424            }
1425            _ => None,
1426        };
1427        Ok(Self {
1428            reader,
1429            schema: Arc::new(schema),
1430            finished: false,
1431            dictionaries_by_id,
1432            projection,
1433            skip_validation: UnsafeFlag::new(),
1434        })
1435    }
1436
1437    /// Deprecated, use [`StreamReader::try_new`] instead.
1438    #[deprecated(since = "53.0.0", note = "use `try_new` instead")]
1439    pub fn try_new_unbuffered(
1440        reader: R,
1441        projection: Option<Vec<usize>>,
1442    ) -> Result<Self, ArrowError> {
1443        Self::try_new(reader, projection)
1444    }
1445
1446    /// Return the schema of the stream
1447    pub fn schema(&self) -> SchemaRef {
1448        self.schema.clone()
1449    }
1450
1451    /// Check if the stream is finished
1452    pub fn is_finished(&self) -> bool {
1453        self.finished
1454    }
1455
1456    fn maybe_next(&mut self) -> Result<Option<RecordBatch>, ArrowError> {
1457        if self.finished {
1458            return Ok(None);
1459        }
1460        // determine metadata length
1461        let mut meta_size: [u8; 4] = [0; 4];
1462
1463        match self.reader.read_exact(&mut meta_size) {
1464            Ok(()) => (),
1465            Err(e) => {
1466                return if e.kind() == std::io::ErrorKind::UnexpectedEof {
1467                    // Handle EOF without the "0xFFFFFFFF 0x00000000"
1468                    // valid according to:
1469                    // https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format
1470                    self.finished = true;
1471                    Ok(None)
1472                } else {
1473                    Err(ArrowError::from(e))
1474                };
1475            }
1476        }
1477
1478        let meta_len = {
1479            // If a continuation marker is encountered, skip over it and read
1480            // the size from the next four bytes.
1481            if meta_size == CONTINUATION_MARKER {
1482                self.reader.read_exact(&mut meta_size)?;
1483            }
1484            i32::from_le_bytes(meta_size)
1485        };
1486
1487        if meta_len == 0 {
1488            // the stream has ended, mark the reader as finished
1489            self.finished = true;
1490            return Ok(None);
1491        }
1492
1493        let mut meta_buffer = vec![0; meta_len as usize];
1494        self.reader.read_exact(&mut meta_buffer)?;
1495
1496        let vecs = &meta_buffer.to_vec();
1497        let message = crate::root_as_message(vecs).map_err(|err| {
1498            ArrowError::ParseError(format!("Unable to get root as message: {err:?}"))
1499        })?;
1500
1501        match message.header_type() {
1502            crate::MessageHeader::Schema => Err(ArrowError::IpcError(
1503                "Not expecting a schema when messages are read".to_string(),
1504            )),
1505            crate::MessageHeader::RecordBatch => {
1506                let batch = message.header_as_record_batch().ok_or_else(|| {
1507                    ArrowError::IpcError("Unable to read IPC message as record batch".to_string())
1508                })?;
1509                // read the block that makes up the record batch into a buffer
1510                let mut buf = MutableBuffer::from_len_zeroed(message.bodyLength() as usize);
1511                self.reader.read_exact(&mut buf)?;
1512
1513                RecordBatchDecoder::try_new(
1514                    &buf.into(),
1515                    batch,
1516                    self.schema(),
1517                    &self.dictionaries_by_id,
1518                    &message.version(),
1519                )?
1520                .with_projection(self.projection.as_ref().map(|x| x.0.as_ref()))
1521                .with_require_alignment(false)
1522                .with_skip_validation(self.skip_validation.clone())
1523                .read_record_batch()
1524                .map(Some)
1525            }
1526            crate::MessageHeader::DictionaryBatch => {
1527                let batch = message.header_as_dictionary_batch().ok_or_else(|| {
1528                    ArrowError::IpcError(
1529                        "Unable to read IPC message as dictionary batch".to_string(),
1530                    )
1531                })?;
1532                // read the block that makes up the dictionary batch into a buffer
1533                let mut buf = MutableBuffer::from_len_zeroed(message.bodyLength() as usize);
1534                self.reader.read_exact(&mut buf)?;
1535
1536                read_dictionary_impl(
1537                    &buf.into(),
1538                    batch,
1539                    &self.schema,
1540                    &mut self.dictionaries_by_id,
1541                    &message.version(),
1542                    false,
1543                    self.skip_validation.clone(),
1544                )?;
1545
1546                // read the next message until we encounter a RecordBatch
1547                self.maybe_next()
1548            }
1549            crate::MessageHeader::NONE => Ok(None),
1550            t => Err(ArrowError::InvalidArgumentError(format!(
1551                "Reading types other than record batches not yet supported, unable to read {t:?} "
1552            ))),
1553        }
1554    }
1555
1556    /// Gets a reference to the underlying reader.
1557    ///
1558    /// It is inadvisable to directly read from the underlying reader.
1559    pub fn get_ref(&self) -> &R {
1560        &self.reader
1561    }
1562
1563    /// Gets a mutable reference to the underlying reader.
1564    ///
1565    /// It is inadvisable to directly read from the underlying reader.
1566    pub fn get_mut(&mut self) -> &mut R {
1567        &mut self.reader
1568    }
1569
1570    /// Specifies if validation should be skipped when reading data (defaults to `false`)
1571    ///
1572    /// # Safety
1573    ///
1574    /// See [`FileDecoder::with_skip_validation`]
1575    pub unsafe fn with_skip_validation(mut self, skip_validation: bool) -> Self {
1576        self.skip_validation.set(skip_validation);
1577        self
1578    }
1579}
1580
1581impl<R: Read> Iterator for StreamReader<R> {
1582    type Item = Result<RecordBatch, ArrowError>;
1583
1584    fn next(&mut self) -> Option<Self::Item> {
1585        self.maybe_next().transpose()
1586    }
1587}
1588
1589impl<R: Read> RecordBatchReader for StreamReader<R> {
1590    fn schema(&self) -> SchemaRef {
1591        self.schema.clone()
1592    }
1593}
1594
1595#[cfg(test)]
1596mod tests {
1597    use crate::convert::fb_to_schema;
1598    use crate::writer::{
1599        unslice_run_array, write_message, DictionaryTracker, IpcDataGenerator, IpcWriteOptions,
1600    };
1601
1602    use super::*;
1603
1604    use crate::{root_as_footer, root_as_message, size_prefixed_root_as_message};
1605    use arrow_array::builder::{PrimitiveRunBuilder, UnionBuilder};
1606    use arrow_array::types::*;
1607    use arrow_buffer::{NullBuffer, OffsetBuffer};
1608    use arrow_data::ArrayDataBuilder;
1609
1610    fn create_test_projection_schema() -> Schema {
1611        // define field types
1612        let list_data_type = DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true)));
1613
1614        let fixed_size_list_data_type =
1615            DataType::FixedSizeList(Arc::new(Field::new_list_field(DataType::Int32, false)), 3);
1616
1617        let union_fields = UnionFields::new(
1618            vec![0, 1],
1619            vec![
1620                Field::new("a", DataType::Int32, false),
1621                Field::new("b", DataType::Float64, false),
1622            ],
1623        );
1624
1625        let union_data_type = DataType::Union(union_fields, UnionMode::Dense);
1626
1627        let struct_fields = Fields::from(vec![
1628            Field::new("id", DataType::Int32, false),
1629            Field::new_list("list", Field::new_list_field(DataType::Int8, true), false),
1630        ]);
1631        let struct_data_type = DataType::Struct(struct_fields);
1632
1633        let run_encoded_data_type = DataType::RunEndEncoded(
1634            Arc::new(Field::new("run_ends", DataType::Int16, false)),
1635            Arc::new(Field::new("values", DataType::Int32, true)),
1636        );
1637
1638        // define schema
1639        Schema::new(vec![
1640            Field::new("f0", DataType::UInt32, false),
1641            Field::new("f1", DataType::Utf8, false),
1642            Field::new("f2", DataType::Boolean, false),
1643            Field::new("f3", union_data_type, true),
1644            Field::new("f4", DataType::Null, true),
1645            Field::new("f5", DataType::Float64, true),
1646            Field::new("f6", list_data_type, false),
1647            Field::new("f7", DataType::FixedSizeBinary(3), true),
1648            Field::new("f8", fixed_size_list_data_type, false),
1649            Field::new("f9", struct_data_type, false),
1650            Field::new("f10", run_encoded_data_type, false),
1651            Field::new("f11", DataType::Boolean, false),
1652            Field::new_dictionary("f12", DataType::Int8, DataType::Utf8, false),
1653            Field::new("f13", DataType::Utf8, false),
1654        ])
1655    }
1656
1657    fn create_test_projection_batch_data(schema: &Schema) -> RecordBatch {
1658        // set test data for each column
1659        let array0 = UInt32Array::from(vec![1, 2, 3]);
1660        let array1 = StringArray::from(vec!["foo", "bar", "baz"]);
1661        let array2 = BooleanArray::from(vec![true, false, true]);
1662
1663        let mut union_builder = UnionBuilder::new_dense();
1664        union_builder.append::<Int32Type>("a", 1).unwrap();
1665        union_builder.append::<Float64Type>("b", 10.1).unwrap();
1666        union_builder.append_null::<Float64Type>("b").unwrap();
1667        let array3 = union_builder.build().unwrap();
1668
1669        let array4 = NullArray::new(3);
1670        let array5 = Float64Array::from(vec![Some(1.1), None, Some(3.3)]);
1671        let array6_values = vec![
1672            Some(vec![Some(10), Some(10), Some(10)]),
1673            Some(vec![Some(20), Some(20), Some(20)]),
1674            Some(vec![Some(30), Some(30)]),
1675        ];
1676        let array6 = ListArray::from_iter_primitive::<Int32Type, _, _>(array6_values);
1677        let array7_values = vec![vec![11, 12, 13], vec![22, 23, 24], vec![33, 34, 35]];
1678        let array7 = FixedSizeBinaryArray::try_from_iter(array7_values.into_iter()).unwrap();
1679
1680        let array8_values = ArrayData::builder(DataType::Int32)
1681            .len(9)
1682            .add_buffer(Buffer::from_slice_ref([40, 41, 42, 43, 44, 45, 46, 47, 48]))
1683            .build()
1684            .unwrap();
1685        let array8_data = ArrayData::builder(schema.field(8).data_type().clone())
1686            .len(3)
1687            .add_child_data(array8_values)
1688            .build()
1689            .unwrap();
1690        let array8 = FixedSizeListArray::from(array8_data);
1691
1692        let array9_id: ArrayRef = Arc::new(Int32Array::from(vec![1001, 1002, 1003]));
1693        let array9_list: ArrayRef =
1694            Arc::new(ListArray::from_iter_primitive::<Int8Type, _, _>(vec![
1695                Some(vec![Some(-10)]),
1696                Some(vec![Some(-20), Some(-20), Some(-20)]),
1697                Some(vec![Some(-30)]),
1698            ]));
1699        let array9 = ArrayDataBuilder::new(schema.field(9).data_type().clone())
1700            .add_child_data(array9_id.into_data())
1701            .add_child_data(array9_list.into_data())
1702            .len(3)
1703            .build()
1704            .unwrap();
1705        let array9: ArrayRef = Arc::new(StructArray::from(array9));
1706
1707        let array10_input = vec![Some(1_i32), None, None];
1708        let mut array10_builder = PrimitiveRunBuilder::<Int16Type, Int32Type>::new();
1709        array10_builder.extend(array10_input);
1710        let array10 = array10_builder.finish();
1711
1712        let array11 = BooleanArray::from(vec![false, false, true]);
1713
1714        let array12_values = StringArray::from(vec!["x", "yy", "zzz"]);
1715        let array12_keys = Int8Array::from_iter_values([1, 1, 2]);
1716        let array12 = DictionaryArray::new(array12_keys, Arc::new(array12_values));
1717
1718        let array13 = StringArray::from(vec!["a", "bb", "ccc"]);
1719
1720        // create record batch
1721        RecordBatch::try_new(
1722            Arc::new(schema.clone()),
1723            vec![
1724                Arc::new(array0),
1725                Arc::new(array1),
1726                Arc::new(array2),
1727                Arc::new(array3),
1728                Arc::new(array4),
1729                Arc::new(array5),
1730                Arc::new(array6),
1731                Arc::new(array7),
1732                Arc::new(array8),
1733                Arc::new(array9),
1734                Arc::new(array10),
1735                Arc::new(array11),
1736                Arc::new(array12),
1737                Arc::new(array13),
1738            ],
1739        )
1740        .unwrap()
1741    }
1742
1743    #[test]
1744    fn test_projection_array_values() {
1745        // define schema
1746        let schema = create_test_projection_schema();
1747
1748        // create record batch with test data
1749        let batch = create_test_projection_batch_data(&schema);
1750
1751        // write record batch in IPC format
1752        let mut buf = Vec::new();
1753        {
1754            let mut writer = crate::writer::FileWriter::try_new(&mut buf, &schema).unwrap();
1755            writer.write(&batch).unwrap();
1756            writer.finish().unwrap();
1757        }
1758
1759        // read record batch with projection
1760        for index in 0..12 {
1761            let projection = vec![index];
1762            let reader = FileReader::try_new(std::io::Cursor::new(buf.clone()), Some(projection));
1763            let read_batch = reader.unwrap().next().unwrap().unwrap();
1764            let projected_column = read_batch.column(0);
1765            let expected_column = batch.column(index);
1766
1767            // check the projected column equals the expected column
1768            assert_eq!(projected_column.as_ref(), expected_column.as_ref());
1769        }
1770
1771        {
1772            // read record batch with reversed projection
1773            let reader =
1774                FileReader::try_new(std::io::Cursor::new(buf.clone()), Some(vec![3, 2, 1]));
1775            let read_batch = reader.unwrap().next().unwrap().unwrap();
1776            let expected_batch = batch.project(&[3, 2, 1]).unwrap();
1777            assert_eq!(read_batch, expected_batch);
1778        }
1779    }
1780
1781    #[test]
1782    fn test_arrow_single_float_row() {
1783        let schema = Schema::new(vec![
1784            Field::new("a", DataType::Float32, false),
1785            Field::new("b", DataType::Float32, false),
1786            Field::new("c", DataType::Int32, false),
1787            Field::new("d", DataType::Int32, false),
1788        ]);
1789        let arrays = vec![
1790            Arc::new(Float32Array::from(vec![1.23])) as ArrayRef,
1791            Arc::new(Float32Array::from(vec![-6.50])) as ArrayRef,
1792            Arc::new(Int32Array::from(vec![2])) as ArrayRef,
1793            Arc::new(Int32Array::from(vec![1])) as ArrayRef,
1794        ];
1795        let batch = RecordBatch::try_new(Arc::new(schema.clone()), arrays).unwrap();
1796        // create stream writer
1797        let mut file = tempfile::tempfile().unwrap();
1798        let mut stream_writer = crate::writer::StreamWriter::try_new(&mut file, &schema).unwrap();
1799        stream_writer.write(&batch).unwrap();
1800        stream_writer.finish().unwrap();
1801
1802        drop(stream_writer);
1803
1804        file.rewind().unwrap();
1805
1806        // read stream back
1807        let reader = StreamReader::try_new(&mut file, None).unwrap();
1808
1809        reader.for_each(|batch| {
1810            let batch = batch.unwrap();
1811            assert!(
1812                batch
1813                    .column(0)
1814                    .as_any()
1815                    .downcast_ref::<Float32Array>()
1816                    .unwrap()
1817                    .value(0)
1818                    != 0.0
1819            );
1820            assert!(
1821                batch
1822                    .column(1)
1823                    .as_any()
1824                    .downcast_ref::<Float32Array>()
1825                    .unwrap()
1826                    .value(0)
1827                    != 0.0
1828            );
1829        });
1830
1831        file.rewind().unwrap();
1832
1833        // Read with projection
1834        let reader = StreamReader::try_new(file, Some(vec![0, 3])).unwrap();
1835
1836        reader.for_each(|batch| {
1837            let batch = batch.unwrap();
1838            assert_eq!(batch.schema().fields().len(), 2);
1839            assert_eq!(batch.schema().fields()[0].data_type(), &DataType::Float32);
1840            assert_eq!(batch.schema().fields()[1].data_type(), &DataType::Int32);
1841        });
1842    }
1843
1844    /// Write the record batch to an in-memory buffer in IPC File format
1845    fn write_ipc(rb: &RecordBatch) -> Vec<u8> {
1846        let mut buf = Vec::new();
1847        let mut writer = crate::writer::FileWriter::try_new(&mut buf, rb.schema_ref()).unwrap();
1848        writer.write(rb).unwrap();
1849        writer.finish().unwrap();
1850        buf
1851    }
1852
1853    /// Return the first record batch read from the IPC File buffer
1854    fn read_ipc(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
1855        let mut reader = FileReader::try_new(std::io::Cursor::new(buf), None)?;
1856        reader.next().unwrap()
1857    }
1858
1859    /// Return the first record batch read from the IPC File buffer, disabling
1860    /// validation
1861    fn read_ipc_skip_validation(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
1862        let mut reader = unsafe {
1863            FileReader::try_new(std::io::Cursor::new(buf), None)?.with_skip_validation(true)
1864        };
1865        reader.next().unwrap()
1866    }
1867
1868    fn roundtrip_ipc(rb: &RecordBatch) -> RecordBatch {
1869        let buf = write_ipc(rb);
1870        read_ipc(&buf).unwrap()
1871    }
1872
1873    /// Return the first record batch read from the IPC File buffer
1874    /// using the FileDecoder API
1875    fn read_ipc_with_decoder(buf: Vec<u8>) -> Result<RecordBatch, ArrowError> {
1876        read_ipc_with_decoder_inner(buf, false)
1877    }
1878
1879    /// Return the first record batch read from the IPC File buffer
1880    /// using the FileDecoder API, disabling validation
1881    fn read_ipc_with_decoder_skip_validation(buf: Vec<u8>) -> Result<RecordBatch, ArrowError> {
1882        read_ipc_with_decoder_inner(buf, true)
1883    }
1884
1885    fn read_ipc_with_decoder_inner(
1886        buf: Vec<u8>,
1887        skip_validation: bool,
1888    ) -> Result<RecordBatch, ArrowError> {
1889        let buffer = Buffer::from_vec(buf);
1890        let trailer_start = buffer.len() - 10;
1891        let footer_len = read_footer_length(buffer[trailer_start..].try_into().unwrap())?;
1892        let footer = root_as_footer(&buffer[trailer_start - footer_len..trailer_start])
1893            .map_err(|e| ArrowError::InvalidArgumentError(format!("Invalid footer: {e}")))?;
1894
1895        let schema = fb_to_schema(footer.schema().unwrap());
1896
1897        let mut decoder = unsafe {
1898            FileDecoder::new(Arc::new(schema), footer.version())
1899                .with_skip_validation(skip_validation)
1900        };
1901        // Read dictionaries
1902        for block in footer.dictionaries().iter().flatten() {
1903            let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
1904            let data = buffer.slice_with_length(block.offset() as _, block_len);
1905            decoder.read_dictionary(block, &data)?
1906        }
1907
1908        // Read record batch
1909        let batches = footer.recordBatches().unwrap();
1910        assert_eq!(batches.len(), 1); // Only wrote a single batch
1911
1912        let block = batches.get(0);
1913        let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
1914        let data = buffer.slice_with_length(block.offset() as _, block_len);
1915        Ok(decoder.read_record_batch(block, &data)?.unwrap())
1916    }
1917
1918    /// Write the record batch to an in-memory buffer in IPC Stream format
1919    fn write_stream(rb: &RecordBatch) -> Vec<u8> {
1920        let mut buf = Vec::new();
1921        let mut writer = crate::writer::StreamWriter::try_new(&mut buf, rb.schema_ref()).unwrap();
1922        writer.write(rb).unwrap();
1923        writer.finish().unwrap();
1924        buf
1925    }
1926
1927    /// Return the first record batch read from the IPC Stream buffer
1928    fn read_stream(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
1929        let mut reader = StreamReader::try_new(std::io::Cursor::new(buf), None)?;
1930        reader.next().unwrap()
1931    }
1932
1933    /// Return the first record batch read from the IPC Stream buffer,
1934    /// disabling validation
1935    fn read_stream_skip_validation(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
1936        let mut reader = unsafe {
1937            StreamReader::try_new(std::io::Cursor::new(buf), None)?.with_skip_validation(true)
1938        };
1939        reader.next().unwrap()
1940    }
1941
1942    fn roundtrip_ipc_stream(rb: &RecordBatch) -> RecordBatch {
1943        let buf = write_stream(rb);
1944        read_stream(&buf).unwrap()
1945    }
1946
1947    #[test]
1948    fn test_roundtrip_with_custom_metadata() {
1949        let schema = Schema::new(vec![Field::new("dummy", DataType::Float64, false)]);
1950        let mut buf = Vec::new();
1951        let mut writer = crate::writer::FileWriter::try_new(&mut buf, &schema).unwrap();
1952        let mut test_metadata = HashMap::new();
1953        test_metadata.insert("abc".to_string(), "abc".to_string());
1954        test_metadata.insert("def".to_string(), "def".to_string());
1955        for (k, v) in &test_metadata {
1956            writer.write_metadata(k, v);
1957        }
1958        writer.finish().unwrap();
1959        drop(writer);
1960
1961        let reader = crate::reader::FileReader::try_new(std::io::Cursor::new(buf), None).unwrap();
1962        assert_eq!(reader.custom_metadata(), &test_metadata);
1963    }
1964
1965    #[test]
1966    fn test_roundtrip_nested_dict() {
1967        let inner: DictionaryArray<Int32Type> = vec!["a", "b", "a"].into_iter().collect();
1968
1969        let array = Arc::new(inner) as ArrayRef;
1970
1971        let dctfield = Arc::new(Field::new("dict", array.data_type().clone(), false));
1972
1973        let s = StructArray::from(vec![(dctfield, array)]);
1974        let struct_array = Arc::new(s) as ArrayRef;
1975
1976        let schema = Arc::new(Schema::new(vec![Field::new(
1977            "struct",
1978            struct_array.data_type().clone(),
1979            false,
1980        )]));
1981
1982        let batch = RecordBatch::try_new(schema, vec![struct_array]).unwrap();
1983
1984        assert_eq!(batch, roundtrip_ipc(&batch));
1985    }
1986
1987    #[test]
1988    fn test_roundtrip_nested_dict_no_preserve_dict_id() {
1989        let inner: DictionaryArray<Int32Type> = vec!["a", "b", "a"].into_iter().collect();
1990
1991        let array = Arc::new(inner) as ArrayRef;
1992
1993        let dctfield = Arc::new(Field::new("dict", array.data_type().clone(), false));
1994
1995        let s = StructArray::from(vec![(dctfield, array)]);
1996        let struct_array = Arc::new(s) as ArrayRef;
1997
1998        let schema = Arc::new(Schema::new(vec![Field::new(
1999            "struct",
2000            struct_array.data_type().clone(),
2001            false,
2002        )]));
2003
2004        let batch = RecordBatch::try_new(schema, vec![struct_array]).unwrap();
2005
2006        let mut buf = Vec::new();
2007        let mut writer = crate::writer::FileWriter::try_new_with_options(
2008            &mut buf,
2009            batch.schema_ref(),
2010            IpcWriteOptions::default(),
2011        )
2012        .unwrap();
2013        writer.write(&batch).unwrap();
2014        writer.finish().unwrap();
2015        drop(writer);
2016
2017        let mut reader = FileReader::try_new(std::io::Cursor::new(buf), None).unwrap();
2018
2019        assert_eq!(batch, reader.next().unwrap().unwrap());
2020    }
2021
2022    fn check_union_with_builder(mut builder: UnionBuilder) {
2023        builder.append::<Int32Type>("a", 1).unwrap();
2024        builder.append_null::<Int32Type>("a").unwrap();
2025        builder.append::<Float64Type>("c", 3.0).unwrap();
2026        builder.append::<Int32Type>("a", 4).unwrap();
2027        builder.append::<Int64Type>("d", 11).unwrap();
2028        let union = builder.build().unwrap();
2029
2030        let schema = Arc::new(Schema::new(vec![Field::new(
2031            "union",
2032            union.data_type().clone(),
2033            false,
2034        )]));
2035
2036        let union_array = Arc::new(union) as ArrayRef;
2037
2038        let rb = RecordBatch::try_new(schema, vec![union_array]).unwrap();
2039        let rb2 = roundtrip_ipc(&rb);
2040        // TODO: equality not yet implemented for union, so we check that the length of the array is
2041        // the same and that all of the buffers are the same instead.
2042        assert_eq!(rb.schema(), rb2.schema());
2043        assert_eq!(rb.num_columns(), rb2.num_columns());
2044        assert_eq!(rb.num_rows(), rb2.num_rows());
2045        let union1 = rb.column(0);
2046        let union2 = rb2.column(0);
2047
2048        assert_eq!(union1, union2);
2049    }
2050
2051    #[test]
2052    fn test_roundtrip_dense_union() {
2053        check_union_with_builder(UnionBuilder::new_dense());
2054    }
2055
2056    #[test]
2057    fn test_roundtrip_sparse_union() {
2058        check_union_with_builder(UnionBuilder::new_sparse());
2059    }
2060
2061    #[test]
2062    fn test_roundtrip_struct_empty_fields() {
2063        let nulls = NullBuffer::from(&[true, true, false]);
2064        let rb = RecordBatch::try_from_iter([(
2065            "",
2066            Arc::new(StructArray::new_empty_fields(nulls.len(), Some(nulls))) as _,
2067        )])
2068        .unwrap();
2069        let rb2 = roundtrip_ipc(&rb);
2070        assert_eq!(rb, rb2);
2071    }
2072
2073    #[test]
2074    fn test_roundtrip_stream_run_array_sliced() {
2075        let run_array_1: Int32RunArray = vec!["a", "a", "a", "b", "b", "c", "c", "c"]
2076            .into_iter()
2077            .collect();
2078        let run_array_1_sliced = run_array_1.slice(2, 5);
2079
2080        let run_array_2_inupt = vec![Some(1_i32), None, None, Some(2), Some(2)];
2081        let mut run_array_2_builder = PrimitiveRunBuilder::<Int16Type, Int32Type>::new();
2082        run_array_2_builder.extend(run_array_2_inupt);
2083        let run_array_2 = run_array_2_builder.finish();
2084
2085        let schema = Arc::new(Schema::new(vec![
2086            Field::new(
2087                "run_array_1_sliced",
2088                run_array_1_sliced.data_type().clone(),
2089                false,
2090            ),
2091            Field::new("run_array_2", run_array_2.data_type().clone(), false),
2092        ]));
2093        let input_batch = RecordBatch::try_new(
2094            schema,
2095            vec![Arc::new(run_array_1_sliced.clone()), Arc::new(run_array_2)],
2096        )
2097        .unwrap();
2098        let output_batch = roundtrip_ipc_stream(&input_batch);
2099
2100        // As partial comparison not yet supported for run arrays, the sliced run array
2101        // has to be unsliced before comparing with the output. the second run array
2102        // can be compared as such.
2103        assert_eq!(input_batch.column(1), output_batch.column(1));
2104
2105        let run_array_1_unsliced = unslice_run_array(run_array_1_sliced.into_data()).unwrap();
2106        assert_eq!(run_array_1_unsliced, output_batch.column(0).into_data());
2107    }
2108
2109    #[test]
2110    fn test_roundtrip_stream_nested_dict() {
2111        let xs = vec!["AA", "BB", "AA", "CC", "BB"];
2112        let dict = Arc::new(
2113            xs.clone()
2114                .into_iter()
2115                .collect::<DictionaryArray<Int8Type>>(),
2116        );
2117        let string_array: ArrayRef = Arc::new(StringArray::from(xs.clone()));
2118        let struct_array = StructArray::from(vec![
2119            (
2120                Arc::new(Field::new("f2.1", DataType::Utf8, false)),
2121                string_array,
2122            ),
2123            (
2124                Arc::new(Field::new("f2.2_struct", dict.data_type().clone(), false)),
2125                dict.clone() as ArrayRef,
2126            ),
2127        ]);
2128        let schema = Arc::new(Schema::new(vec![
2129            Field::new("f1_string", DataType::Utf8, false),
2130            Field::new("f2_struct", struct_array.data_type().clone(), false),
2131        ]));
2132        let input_batch = RecordBatch::try_new(
2133            schema,
2134            vec![
2135                Arc::new(StringArray::from(xs.clone())),
2136                Arc::new(struct_array),
2137            ],
2138        )
2139        .unwrap();
2140        let output_batch = roundtrip_ipc_stream(&input_batch);
2141        assert_eq!(input_batch, output_batch);
2142    }
2143
2144    #[test]
2145    fn test_roundtrip_stream_nested_dict_of_map_of_dict() {
2146        let values = StringArray::from(vec![Some("a"), None, Some("b"), Some("c")]);
2147        let values = Arc::new(values) as ArrayRef;
2148        let value_dict_keys = Int8Array::from_iter_values([0, 1, 1, 2, 3, 1]);
2149        let value_dict_array = DictionaryArray::new(value_dict_keys, values.clone());
2150
2151        let key_dict_keys = Int8Array::from_iter_values([0, 0, 2, 1, 1, 3]);
2152        let key_dict_array = DictionaryArray::new(key_dict_keys, values);
2153
2154        #[allow(deprecated)]
2155        let keys_field = Arc::new(Field::new_dict(
2156            "keys",
2157            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
2158            true, // It is technically not legal for this field to be null.
2159            1,
2160            false,
2161        ));
2162        #[allow(deprecated)]
2163        let values_field = Arc::new(Field::new_dict(
2164            "values",
2165            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
2166            true,
2167            2,
2168            false,
2169        ));
2170        let entry_struct = StructArray::from(vec![
2171            (keys_field, make_array(key_dict_array.into_data())),
2172            (values_field, make_array(value_dict_array.into_data())),
2173        ]);
2174        let map_data_type = DataType::Map(
2175            Arc::new(Field::new(
2176                "entries",
2177                entry_struct.data_type().clone(),
2178                false,
2179            )),
2180            false,
2181        );
2182
2183        let entry_offsets = Buffer::from_slice_ref([0, 2, 4, 6]);
2184        let map_data = ArrayData::builder(map_data_type)
2185            .len(3)
2186            .add_buffer(entry_offsets)
2187            .add_child_data(entry_struct.into_data())
2188            .build()
2189            .unwrap();
2190        let map_array = MapArray::from(map_data);
2191
2192        let dict_keys = Int8Array::from_iter_values([0, 1, 1, 2, 2, 1]);
2193        let dict_dict_array = DictionaryArray::new(dict_keys, Arc::new(map_array));
2194
2195        let schema = Arc::new(Schema::new(vec![Field::new(
2196            "f1",
2197            dict_dict_array.data_type().clone(),
2198            false,
2199        )]));
2200        let input_batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
2201        let output_batch = roundtrip_ipc_stream(&input_batch);
2202        assert_eq!(input_batch, output_batch);
2203    }
2204
2205    fn test_roundtrip_stream_dict_of_list_of_dict_impl<
2206        OffsetSize: OffsetSizeTrait,
2207        U: ArrowNativeType,
2208    >(
2209        list_data_type: DataType,
2210        offsets: &[U; 5],
2211    ) {
2212        let values = StringArray::from(vec![Some("a"), None, Some("c"), None]);
2213        let keys = Int8Array::from_iter_values([0, 0, 1, 2, 0, 1, 3]);
2214        let dict_array = DictionaryArray::new(keys, Arc::new(values));
2215        let dict_data = dict_array.to_data();
2216
2217        let value_offsets = Buffer::from_slice_ref(offsets);
2218
2219        let list_data = ArrayData::builder(list_data_type)
2220            .len(4)
2221            .add_buffer(value_offsets)
2222            .add_child_data(dict_data)
2223            .build()
2224            .unwrap();
2225        let list_array = GenericListArray::<OffsetSize>::from(list_data);
2226
2227        let keys_for_dict = Int8Array::from_iter_values([0, 3, 0, 1, 1, 2, 0, 1, 3]);
2228        let dict_dict_array = DictionaryArray::new(keys_for_dict, Arc::new(list_array));
2229
2230        let schema = Arc::new(Schema::new(vec![Field::new(
2231            "f1",
2232            dict_dict_array.data_type().clone(),
2233            false,
2234        )]));
2235        let input_batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
2236        let output_batch = roundtrip_ipc_stream(&input_batch);
2237        assert_eq!(input_batch, output_batch);
2238    }
2239
2240    #[test]
2241    fn test_roundtrip_stream_dict_of_list_of_dict() {
2242        // list
2243        #[allow(deprecated)]
2244        let list_data_type = DataType::List(Arc::new(Field::new_dict(
2245            "item",
2246            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
2247            true,
2248            1,
2249            false,
2250        )));
2251        let offsets: &[i32; 5] = &[0, 2, 4, 4, 6];
2252        test_roundtrip_stream_dict_of_list_of_dict_impl::<i32, i32>(list_data_type, offsets);
2253
2254        // large list
2255        #[allow(deprecated)]
2256        let list_data_type = DataType::LargeList(Arc::new(Field::new_dict(
2257            "item",
2258            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
2259            true,
2260            1,
2261            false,
2262        )));
2263        let offsets: &[i64; 5] = &[0, 2, 4, 4, 7];
2264        test_roundtrip_stream_dict_of_list_of_dict_impl::<i64, i64>(list_data_type, offsets);
2265    }
2266
2267    #[test]
2268    fn test_roundtrip_stream_dict_of_fixed_size_list_of_dict() {
2269        let values = StringArray::from(vec![Some("a"), None, Some("c"), None]);
2270        let keys = Int8Array::from_iter_values([0, 0, 1, 2, 0, 1, 3, 1, 2]);
2271        let dict_array = DictionaryArray::new(keys, Arc::new(values));
2272        let dict_data = dict_array.into_data();
2273
2274        #[allow(deprecated)]
2275        let list_data_type = DataType::FixedSizeList(
2276            Arc::new(Field::new_dict(
2277                "item",
2278                DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
2279                true,
2280                1,
2281                false,
2282            )),
2283            3,
2284        );
2285        let list_data = ArrayData::builder(list_data_type)
2286            .len(3)
2287            .add_child_data(dict_data)
2288            .build()
2289            .unwrap();
2290        let list_array = FixedSizeListArray::from(list_data);
2291
2292        let keys_for_dict = Int8Array::from_iter_values([0, 1, 0, 1, 1, 2, 0, 1, 2]);
2293        let dict_dict_array = DictionaryArray::new(keys_for_dict, Arc::new(list_array));
2294
2295        let schema = Arc::new(Schema::new(vec![Field::new(
2296            "f1",
2297            dict_dict_array.data_type().clone(),
2298            false,
2299        )]));
2300        let input_batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
2301        let output_batch = roundtrip_ipc_stream(&input_batch);
2302        assert_eq!(input_batch, output_batch);
2303    }
2304
2305    const LONG_TEST_STRING: &str =
2306        "This is a long string to make sure binary view array handles it";
2307
2308    #[test]
2309    fn test_roundtrip_view_types() {
2310        let schema = Schema::new(vec![
2311            Field::new("field_1", DataType::BinaryView, true),
2312            Field::new("field_2", DataType::Utf8, true),
2313            Field::new("field_3", DataType::Utf8View, true),
2314        ]);
2315        let bin_values: Vec<Option<&[u8]>> = vec![
2316            Some(b"foo"),
2317            None,
2318            Some(b"bar"),
2319            Some(LONG_TEST_STRING.as_bytes()),
2320        ];
2321        let utf8_values: Vec<Option<&str>> =
2322            vec![Some("foo"), None, Some("bar"), Some(LONG_TEST_STRING)];
2323        let bin_view_array = BinaryViewArray::from_iter(bin_values);
2324        let utf8_array = StringArray::from_iter(utf8_values.iter());
2325        let utf8_view_array = StringViewArray::from_iter(utf8_values);
2326        let record_batch = RecordBatch::try_new(
2327            Arc::new(schema.clone()),
2328            vec![
2329                Arc::new(bin_view_array),
2330                Arc::new(utf8_array),
2331                Arc::new(utf8_view_array),
2332            ],
2333        )
2334        .unwrap();
2335
2336        assert_eq!(record_batch, roundtrip_ipc(&record_batch));
2337        assert_eq!(record_batch, roundtrip_ipc_stream(&record_batch));
2338
2339        let sliced_batch = record_batch.slice(1, 2);
2340        assert_eq!(sliced_batch, roundtrip_ipc(&sliced_batch));
2341        assert_eq!(sliced_batch, roundtrip_ipc_stream(&sliced_batch));
2342    }
2343
2344    #[test]
2345    fn test_roundtrip_view_types_nested_dict() {
2346        let bin_values: Vec<Option<&[u8]>> = vec![
2347            Some(b"foo"),
2348            None,
2349            Some(b"bar"),
2350            Some(LONG_TEST_STRING.as_bytes()),
2351            Some(b"field"),
2352        ];
2353        let utf8_values: Vec<Option<&str>> = vec![
2354            Some("foo"),
2355            None,
2356            Some("bar"),
2357            Some(LONG_TEST_STRING),
2358            Some("field"),
2359        ];
2360        let bin_view_array = Arc::new(BinaryViewArray::from_iter(bin_values));
2361        let utf8_view_array = Arc::new(StringViewArray::from_iter(utf8_values));
2362
2363        let key_dict_keys = Int8Array::from_iter_values([0, 0, 1, 2, 0, 1, 3]);
2364        let key_dict_array = DictionaryArray::new(key_dict_keys, utf8_view_array.clone());
2365        #[allow(deprecated)]
2366        let keys_field = Arc::new(Field::new_dict(
2367            "keys",
2368            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8View)),
2369            true,
2370            1,
2371            false,
2372        ));
2373
2374        let value_dict_keys = Int8Array::from_iter_values([0, 3, 0, 1, 2, 0, 1]);
2375        let value_dict_array = DictionaryArray::new(value_dict_keys, bin_view_array);
2376        #[allow(deprecated)]
2377        let values_field = Arc::new(Field::new_dict(
2378            "values",
2379            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::BinaryView)),
2380            true,
2381            2,
2382            false,
2383        ));
2384        let entry_struct = StructArray::from(vec![
2385            (keys_field, make_array(key_dict_array.into_data())),
2386            (values_field, make_array(value_dict_array.into_data())),
2387        ]);
2388
2389        let map_data_type = DataType::Map(
2390            Arc::new(Field::new(
2391                "entries",
2392                entry_struct.data_type().clone(),
2393                false,
2394            )),
2395            false,
2396        );
2397        let entry_offsets = Buffer::from_slice_ref([0, 2, 4, 7]);
2398        let map_data = ArrayData::builder(map_data_type)
2399            .len(3)
2400            .add_buffer(entry_offsets)
2401            .add_child_data(entry_struct.into_data())
2402            .build()
2403            .unwrap();
2404        let map_array = MapArray::from(map_data);
2405
2406        let dict_keys = Int8Array::from_iter_values([0, 1, 0, 1, 1, 2, 0, 1, 2]);
2407        let dict_dict_array = DictionaryArray::new(dict_keys, Arc::new(map_array));
2408        let schema = Arc::new(Schema::new(vec![Field::new(
2409            "f1",
2410            dict_dict_array.data_type().clone(),
2411            false,
2412        )]));
2413        let batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
2414        assert_eq!(batch, roundtrip_ipc(&batch));
2415        assert_eq!(batch, roundtrip_ipc_stream(&batch));
2416
2417        let sliced_batch = batch.slice(1, 2);
2418        assert_eq!(sliced_batch, roundtrip_ipc(&sliced_batch));
2419        assert_eq!(sliced_batch, roundtrip_ipc_stream(&sliced_batch));
2420    }
2421
2422    #[test]
2423    fn test_no_columns_batch() {
2424        let schema = Arc::new(Schema::empty());
2425        let options = RecordBatchOptions::new()
2426            .with_match_field_names(true)
2427            .with_row_count(Some(10));
2428        let input_batch = RecordBatch::try_new_with_options(schema, vec![], &options).unwrap();
2429        let output_batch = roundtrip_ipc_stream(&input_batch);
2430        assert_eq!(input_batch, output_batch);
2431    }
2432
2433    #[test]
2434    fn test_unaligned() {
2435        let batch = RecordBatch::try_from_iter(vec![(
2436            "i32",
2437            Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _,
2438        )])
2439        .unwrap();
2440
2441        let gen = IpcDataGenerator {};
2442        let mut dict_tracker = DictionaryTracker::new(false);
2443        let (_, encoded) = gen
2444            .encoded_batch(&batch, &mut dict_tracker, &Default::default())
2445            .unwrap();
2446
2447        let message = root_as_message(&encoded.ipc_message).unwrap();
2448
2449        // Construct an unaligned buffer
2450        let mut buffer = MutableBuffer::with_capacity(encoded.arrow_data.len() + 1);
2451        buffer.push(0_u8);
2452        buffer.extend_from_slice(&encoded.arrow_data);
2453        let b = Buffer::from(buffer).slice(1);
2454        assert_ne!(b.as_ptr().align_offset(8), 0);
2455
2456        let ipc_batch = message.header_as_record_batch().unwrap();
2457        let roundtrip = RecordBatchDecoder::try_new(
2458            &b,
2459            ipc_batch,
2460            batch.schema(),
2461            &Default::default(),
2462            &message.version(),
2463        )
2464        .unwrap()
2465        .with_require_alignment(false)
2466        .read_record_batch()
2467        .unwrap();
2468        assert_eq!(batch, roundtrip);
2469    }
2470
2471    #[test]
2472    fn test_unaligned_throws_error_with_require_alignment() {
2473        let batch = RecordBatch::try_from_iter(vec![(
2474            "i32",
2475            Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _,
2476        )])
2477        .unwrap();
2478
2479        let gen = IpcDataGenerator {};
2480        let mut dict_tracker = DictionaryTracker::new(false);
2481        let (_, encoded) = gen
2482            .encoded_batch(&batch, &mut dict_tracker, &Default::default())
2483            .unwrap();
2484
2485        let message = root_as_message(&encoded.ipc_message).unwrap();
2486
2487        // Construct an unaligned buffer
2488        let mut buffer = MutableBuffer::with_capacity(encoded.arrow_data.len() + 1);
2489        buffer.push(0_u8);
2490        buffer.extend_from_slice(&encoded.arrow_data);
2491        let b = Buffer::from(buffer).slice(1);
2492        assert_ne!(b.as_ptr().align_offset(8), 0);
2493
2494        let ipc_batch = message.header_as_record_batch().unwrap();
2495        let result = RecordBatchDecoder::try_new(
2496            &b,
2497            ipc_batch,
2498            batch.schema(),
2499            &Default::default(),
2500            &message.version(),
2501        )
2502        .unwrap()
2503        .with_require_alignment(true)
2504        .read_record_batch();
2505
2506        let error = result.unwrap_err();
2507        assert_eq!(
2508            error.to_string(),
2509            "Invalid argument error: Misaligned buffers[0] in array of type Int32, \
2510             offset from expected alignment of 4 by 1"
2511        );
2512    }
2513
2514    #[test]
2515    fn test_file_with_massive_column_count() {
2516        // 499_999 is upper limit for default settings (1_000_000)
2517        let limit = 600_000;
2518
2519        let fields = (0..limit)
2520            .map(|i| Field::new(format!("{i}"), DataType::Boolean, false))
2521            .collect::<Vec<_>>();
2522        let schema = Arc::new(Schema::new(fields));
2523        let batch = RecordBatch::new_empty(schema);
2524
2525        let mut buf = Vec::new();
2526        let mut writer = crate::writer::FileWriter::try_new(&mut buf, batch.schema_ref()).unwrap();
2527        writer.write(&batch).unwrap();
2528        writer.finish().unwrap();
2529        drop(writer);
2530
2531        let mut reader = FileReaderBuilder::new()
2532            .with_max_footer_fb_tables(1_500_000)
2533            .build(std::io::Cursor::new(buf))
2534            .unwrap();
2535        let roundtrip_batch = reader.next().unwrap().unwrap();
2536
2537        assert_eq!(batch, roundtrip_batch);
2538    }
2539
2540    #[test]
2541    fn test_file_with_deeply_nested_columns() {
2542        // 60 is upper limit for default settings (64)
2543        let limit = 61;
2544
2545        let fields = (0..limit).fold(
2546            vec![Field::new("leaf", DataType::Boolean, false)],
2547            |field, index| vec![Field::new_struct(format!("{index}"), field, false)],
2548        );
2549        let schema = Arc::new(Schema::new(fields));
2550        let batch = RecordBatch::new_empty(schema);
2551
2552        let mut buf = Vec::new();
2553        let mut writer = crate::writer::FileWriter::try_new(&mut buf, batch.schema_ref()).unwrap();
2554        writer.write(&batch).unwrap();
2555        writer.finish().unwrap();
2556        drop(writer);
2557
2558        let mut reader = FileReaderBuilder::new()
2559            .with_max_footer_fb_depth(65)
2560            .build(std::io::Cursor::new(buf))
2561            .unwrap();
2562        let roundtrip_batch = reader.next().unwrap().unwrap();
2563
2564        assert_eq!(batch, roundtrip_batch);
2565    }
2566
2567    #[test]
2568    fn test_invalid_struct_array_ipc_read_errors() {
2569        let a_field = Field::new("a", DataType::Int32, false);
2570        let b_field = Field::new("b", DataType::Int32, false);
2571        let struct_fields = Fields::from(vec![a_field.clone(), b_field.clone()]);
2572
2573        let a_array_data = ArrayData::builder(a_field.data_type().clone())
2574            .len(4)
2575            .add_buffer(Buffer::from_slice_ref([1, 2, 3, 4]))
2576            .build()
2577            .unwrap();
2578        let b_array_data = ArrayData::builder(b_field.data_type().clone())
2579            .len(3)
2580            .add_buffer(Buffer::from_slice_ref([5, 6, 7]))
2581            .build()
2582            .unwrap();
2583
2584        let invalid_struct_arr = unsafe {
2585            StructArray::new_unchecked(
2586                struct_fields,
2587                vec![make_array(a_array_data), make_array(b_array_data)],
2588                None,
2589            )
2590        };
2591
2592        expect_ipc_validation_error(
2593            Arc::new(invalid_struct_arr),
2594            "Invalid argument error: Incorrect array length for StructArray field \"b\", expected 4 got 3",
2595        );
2596    }
2597
2598    #[test]
2599    fn test_invalid_nested_array_ipc_read_errors() {
2600        // one of the nested arrays has invalid data
2601        let a_field = Field::new("a", DataType::Int32, false);
2602        let b_field = Field::new("b", DataType::Utf8, false);
2603
2604        let schema = Arc::new(Schema::new(vec![Field::new_struct(
2605            "s",
2606            vec![a_field.clone(), b_field.clone()],
2607            false,
2608        )]));
2609
2610        let a_array_data = ArrayData::builder(a_field.data_type().clone())
2611            .len(4)
2612            .add_buffer(Buffer::from_slice_ref([1, 2, 3, 4]))
2613            .build()
2614            .unwrap();
2615        // invalid nested child array -- length is correct, but has invalid utf8 data
2616        let b_array_data = {
2617            let valid: &[u8] = b"   ";
2618            let mut invalid = vec![];
2619            invalid.extend_from_slice(b"ValidString");
2620            invalid.extend_from_slice(INVALID_UTF8_FIRST_CHAR);
2621            let binary_array =
2622                BinaryArray::from_iter(vec![None, Some(valid), None, Some(&invalid)]);
2623            let array = unsafe {
2624                StringArray::new_unchecked(
2625                    binary_array.offsets().clone(),
2626                    binary_array.values().clone(),
2627                    binary_array.nulls().cloned(),
2628                )
2629            };
2630            array.into_data()
2631        };
2632        let struct_data_type = schema.field(0).data_type();
2633
2634        let invalid_struct_arr = unsafe {
2635            make_array(
2636                ArrayData::builder(struct_data_type.clone())
2637                    .len(4)
2638                    .add_child_data(a_array_data)
2639                    .add_child_data(b_array_data)
2640                    .build_unchecked(),
2641            )
2642        };
2643        expect_ipc_validation_error(
2644            Arc::new(invalid_struct_arr),
2645            "Invalid argument error: Invalid UTF8 sequence at string index 3 (3..18): invalid utf-8 sequence of 1 bytes from index 11",
2646        );
2647    }
2648
2649    #[test]
2650    fn test_same_dict_id_without_preserve() {
2651        let batch = RecordBatch::try_new(
2652            Arc::new(Schema::new(
2653                ["a", "b"]
2654                    .iter()
2655                    .map(|name| {
2656                        #[allow(deprecated)]
2657                        Field::new_dict(
2658                            name.to_string(),
2659                            DataType::Dictionary(
2660                                Box::new(DataType::Int32),
2661                                Box::new(DataType::Utf8),
2662                            ),
2663                            true,
2664                            0,
2665                            false,
2666                        )
2667                    })
2668                    .collect::<Vec<Field>>(),
2669            )),
2670            vec![
2671                Arc::new(
2672                    vec![Some("c"), Some("d")]
2673                        .into_iter()
2674                        .collect::<DictionaryArray<Int32Type>>(),
2675                ) as ArrayRef,
2676                Arc::new(
2677                    vec![Some("e"), Some("f")]
2678                        .into_iter()
2679                        .collect::<DictionaryArray<Int32Type>>(),
2680                ) as ArrayRef,
2681            ],
2682        )
2683        .expect("Failed to create RecordBatch");
2684
2685        // serialize the record batch as an IPC stream
2686        let mut buf = vec![];
2687        {
2688            let mut writer = crate::writer::StreamWriter::try_new_with_options(
2689                &mut buf,
2690                batch.schema().as_ref(),
2691                crate::writer::IpcWriteOptions::default(),
2692            )
2693            .expect("Failed to create StreamWriter");
2694            writer.write(&batch).expect("Failed to write RecordBatch");
2695            writer.finish().expect("Failed to finish StreamWriter");
2696        }
2697
2698        StreamReader::try_new(std::io::Cursor::new(buf), None)
2699            .expect("Failed to create StreamReader")
2700            .for_each(|decoded_batch| {
2701                assert_eq!(decoded_batch.expect("Failed to read RecordBatch"), batch);
2702            });
2703    }
2704
2705    #[test]
2706    fn test_validation_of_invalid_list_array() {
2707        // ListArray with invalid offsets
2708        let array = unsafe {
2709            let values = Int32Array::from(vec![1, 2, 3]);
2710            let bad_offsets = ScalarBuffer::<i32>::from(vec![0, 2, 4, 2]); // offsets can't go backwards
2711            let offsets = OffsetBuffer::new_unchecked(bad_offsets); // INVALID array created
2712            let field = Field::new_list_field(DataType::Int32, true);
2713            let nulls = None;
2714            ListArray::new(Arc::new(field), offsets, Arc::new(values), nulls)
2715        };
2716
2717        expect_ipc_validation_error(
2718            Arc::new(array),
2719            "Invalid argument error: Offset invariant failure: offset at position 2 out of bounds: 4 > 2"
2720        );
2721    }
2722
2723    #[test]
2724    fn test_validation_of_invalid_string_array() {
2725        let valid: &[u8] = b"   ";
2726        let mut invalid = vec![];
2727        invalid.extend_from_slice(b"ThisStringIsCertainlyLongerThan12Bytes");
2728        invalid.extend_from_slice(INVALID_UTF8_FIRST_CHAR);
2729        let binary_array = BinaryArray::from_iter(vec![None, Some(valid), None, Some(&invalid)]);
2730        // data is not valid utf8 we can not construct a correct StringArray
2731        // safely, so purposely create an invalid StringArray
2732        let array = unsafe {
2733            StringArray::new_unchecked(
2734                binary_array.offsets().clone(),
2735                binary_array.values().clone(),
2736                binary_array.nulls().cloned(),
2737            )
2738        };
2739        expect_ipc_validation_error(
2740            Arc::new(array),
2741            "Invalid argument error: Invalid UTF8 sequence at string index 3 (3..45): invalid utf-8 sequence of 1 bytes from index 38"
2742        );
2743    }
2744
2745    #[test]
2746    fn test_validation_of_invalid_string_view_array() {
2747        let valid: &[u8] = b"   ";
2748        let mut invalid = vec![];
2749        invalid.extend_from_slice(b"ThisStringIsCertainlyLongerThan12Bytes");
2750        invalid.extend_from_slice(INVALID_UTF8_FIRST_CHAR);
2751        let binary_view_array =
2752            BinaryViewArray::from_iter(vec![None, Some(valid), None, Some(&invalid)]);
2753        // data is not valid utf8 we can not construct a correct StringArray
2754        // safely, so purposely create an invalid StringArray
2755        let array = unsafe {
2756            StringViewArray::new_unchecked(
2757                binary_view_array.views().clone(),
2758                binary_view_array.data_buffers().to_vec(),
2759                binary_view_array.nulls().cloned(),
2760            )
2761        };
2762        expect_ipc_validation_error(
2763            Arc::new(array),
2764            "Invalid argument error: Encountered non-UTF-8 data at index 3: invalid utf-8 sequence of 1 bytes from index 38"
2765        );
2766    }
2767
2768    /// return an invalid dictionary array (key is larger than values)
2769    /// ListArray with invalid offsets
2770    #[test]
2771    fn test_validation_of_invalid_dictionary_array() {
2772        let array = unsafe {
2773            let values = StringArray::from_iter_values(["a", "b", "c"]);
2774            let keys = Int32Array::from(vec![1, 200]); // keys are not valid for values
2775            DictionaryArray::new_unchecked(keys, Arc::new(values))
2776        };
2777
2778        expect_ipc_validation_error(
2779            Arc::new(array),
2780            "Invalid argument error: Value at position 1 out of bounds: 200 (should be in [0, 2])",
2781        );
2782    }
2783
2784    #[test]
2785    fn test_validation_of_invalid_union_array() {
2786        let array = unsafe {
2787            let fields = UnionFields::new(
2788                vec![1, 3], // typeids : type id 2 is not valid
2789                vec![
2790                    Field::new("a", DataType::Int32, false),
2791                    Field::new("b", DataType::Utf8, false),
2792                ],
2793            );
2794            let type_ids = ScalarBuffer::from(vec![1i8, 2, 3]); // 2 is invalid
2795            let offsets = None;
2796            let children: Vec<ArrayRef> = vec![
2797                Arc::new(Int32Array::from(vec![10, 20, 30])),
2798                Arc::new(StringArray::from(vec![Some("a"), Some("b"), Some("c")])),
2799            ];
2800
2801            UnionArray::new_unchecked(fields, type_ids, offsets, children)
2802        };
2803
2804        expect_ipc_validation_error(
2805            Arc::new(array),
2806            "Invalid argument error: Type Ids values must match one of the field type ids",
2807        );
2808    }
2809
2810    /// Invalid Utf-8 sequence in the first character
2811    /// <https://stackoverflow.com/questions/1301402/example-invalid-utf8-string>
2812    const INVALID_UTF8_FIRST_CHAR: &[u8] = &[0xa0, 0xa1, 0x20, 0x20];
2813
2814    /// Expect an error when reading the record batch using IPC or IPC Streams
2815    fn expect_ipc_validation_error(array: ArrayRef, expected_err: &str) {
2816        let rb = RecordBatch::try_from_iter([("a", array)]).unwrap();
2817
2818        // IPC Stream format
2819        let buf = write_stream(&rb); // write is ok
2820        read_stream_skip_validation(&buf).unwrap();
2821        let err = read_stream(&buf).unwrap_err();
2822        assert_eq!(err.to_string(), expected_err);
2823
2824        // IPC File format
2825        let buf = write_ipc(&rb); // write is ok
2826        read_ipc_skip_validation(&buf).unwrap();
2827        let err = read_ipc(&buf).unwrap_err();
2828        assert_eq!(err.to_string(), expected_err);
2829
2830        // IPC Format with FileDecoder
2831        read_ipc_with_decoder_skip_validation(buf.clone()).unwrap();
2832        let err = read_ipc_with_decoder(buf).unwrap_err();
2833        assert_eq!(err.to_string(), expected_err);
2834    }
2835
2836    #[test]
2837    fn test_roundtrip_schema() {
2838        let schema = Schema::new(vec![
2839            Field::new(
2840                "a",
2841                DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)),
2842                false,
2843            ),
2844            Field::new(
2845                "b",
2846                DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)),
2847                false,
2848            ),
2849        ]);
2850
2851        let options = IpcWriteOptions::default();
2852        let data_gen = IpcDataGenerator::default();
2853        let mut dict_tracker = DictionaryTracker::new(false);
2854        let encoded_data =
2855            data_gen.schema_to_bytes_with_dictionary_tracker(&schema, &mut dict_tracker, &options);
2856        let mut schema_bytes = vec![];
2857        write_message(&mut schema_bytes, encoded_data, &options).expect("write_message");
2858
2859        let begin_offset: usize = if schema_bytes[0..4].eq(&CONTINUATION_MARKER) {
2860            4
2861        } else {
2862            0
2863        };
2864
2865        size_prefixed_root_as_message(&schema_bytes[begin_offset..])
2866            .expect_err("size_prefixed_root_as_message");
2867
2868        let msg = parse_message(&schema_bytes).expect("parse_message");
2869        let ipc_schema = msg.header_as_schema().expect("header_as_schema");
2870        let new_schema = fb_to_schema(ipc_schema);
2871
2872        assert_eq!(schema, new_schema);
2873    }
2874}