parquet/arrow/arrow_writer/
mod.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Contains writer which writes arrow data into parquet data.
19
20use bytes::Bytes;
21use std::io::{Read, Write};
22use std::iter::Peekable;
23use std::slice::Iter;
24use std::sync::{Arc, Mutex};
25use std::vec::IntoIter;
26
27use arrow_array::cast::AsArray;
28use arrow_array::types::*;
29use arrow_array::{ArrayRef, Int32Array, RecordBatch, RecordBatchWriter};
30use arrow_schema::{
31    ArrowError, DataType as ArrowDataType, Field, IntervalUnit, SchemaRef, TimeUnit,
32};
33
34use super::schema::{add_encoded_arrow_schema_to_metadata, decimal_length_from_precision};
35
36use crate::arrow::ArrowSchemaConverter;
37use crate::arrow::arrow_writer::byte_array::ByteArrayEncoder;
38use crate::column::page::{CompressedPage, PageWriteSpec, PageWriter};
39use crate::column::page_encryption::PageEncryptor;
40use crate::column::writer::encoder::ColumnValueEncoder;
41use crate::column::writer::{
42    ColumnCloseResult, ColumnWriter, GenericColumnWriter, get_column_writer,
43};
44use crate::data_type::{ByteArray, FixedLenByteArray};
45#[cfg(feature = "encryption")]
46use crate::encryption::encrypt::FileEncryptor;
47use crate::errors::{ParquetError, Result};
48use crate::file::metadata::{KeyValue, ParquetMetaData, RowGroupMetaData};
49use crate::file::properties::{WriterProperties, WriterPropertiesPtr};
50use crate::file::reader::{ChunkReader, Length};
51use crate::file::writer::{SerializedFileWriter, SerializedRowGroupWriter};
52use crate::parquet_thrift::{ThriftCompactOutputProtocol, WriteThrift};
53use crate::schema::types::{ColumnDescPtr, SchemaDescPtr, SchemaDescriptor};
54use levels::{ArrayLevels, calculate_array_levels};
55
56mod byte_array;
57mod levels;
58
59/// Encodes [`RecordBatch`] to parquet
60///
61/// Writes Arrow `RecordBatch`es to a Parquet writer. Multiple [`RecordBatch`] will be encoded
62/// to the same row group, up to `max_row_group_size` rows. Any remaining rows will be
63/// flushed on close, leading the final row group in the output file to potentially
64/// contain fewer than `max_row_group_size` rows
65///
66/// # Example: Writing `RecordBatch`es
67/// ```
68/// # use std::sync::Arc;
69/// # use bytes::Bytes;
70/// # use arrow_array::{ArrayRef, Int64Array};
71/// # use arrow_array::RecordBatch;
72/// # use parquet::arrow::arrow_writer::ArrowWriter;
73/// # use parquet::arrow::arrow_reader::ParquetRecordBatchReader;
74/// let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
75/// let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
76///
77/// let mut buffer = Vec::new();
78/// let mut writer = ArrowWriter::try_new(&mut buffer, to_write.schema(), None).unwrap();
79/// writer.write(&to_write).unwrap();
80/// writer.close().unwrap();
81///
82/// let mut reader = ParquetRecordBatchReader::try_new(Bytes::from(buffer), 1024).unwrap();
83/// let read = reader.next().unwrap().unwrap();
84///
85/// assert_eq!(to_write, read);
86/// ```
87///
88/// # Memory Usage and Limiting
89///
90/// The nature of Parquet requires buffering of an entire row group before it can
91/// be flushed to the underlying writer. Data is mostly buffered in its encoded
92/// form, reducing memory usage. However, some data such as dictionary keys,
93/// large strings or very nested data may still result in non-trivial memory
94/// usage.
95///
96/// See Also:
97/// * [`ArrowWriter::memory_size`]: the current memory usage of the writer.
98/// * [`ArrowWriter::in_progress_size`]: Estimated size of the buffered row group,
99///
100/// Call [`Self::flush`] to trigger an early flush of a row group based on a
101/// memory threshold and/or global memory pressure. However,  smaller row groups
102/// result in higher metadata overheads, and thus may worsen compression ratios
103/// and query performance.
104///
105/// ```no_run
106/// # use std::io::Write;
107/// # use arrow_array::RecordBatch;
108/// # use parquet::arrow::ArrowWriter;
109/// # let mut writer: ArrowWriter<Vec<u8>> = todo!();
110/// # let batch: RecordBatch = todo!();
111/// writer.write(&batch).unwrap();
112/// // Trigger an early flush if anticipated size exceeds 1_000_000
113/// if writer.in_progress_size() > 1_000_000 {
114///     writer.flush().unwrap();
115/// }
116/// ```
117///
118/// ## Type Support
119///
120/// The writer supports writing all Arrow [`DataType`]s that have a direct mapping to
121/// Parquet types including  [`StructArray`] and [`ListArray`].
122///
123/// The following are not supported:
124///
125/// * [`IntervalMonthDayNanoArray`]: Parquet does not [support nanosecond intervals].
126///
127/// [`DataType`]: https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html
128/// [`StructArray`]: https://docs.rs/arrow/latest/arrow/array/struct.StructArray.html
129/// [`ListArray`]: https://docs.rs/arrow/latest/arrow/array/type.ListArray.html
130/// [`IntervalMonthDayNanoArray`]: https://docs.rs/arrow/latest/arrow/array/type.IntervalMonthDayNanoArray.html
131/// [support nanosecond intervals]: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#interval
132///
133/// ## Type Compatibility
134/// The writer can write Arrow [`RecordBatch`]s that are logically equivalent. This means that for
135/// a  given column, the writer can accept multiple Arrow [`DataType`]s that contain the same
136/// value type.
137///
138/// For example, the following [`DataType`]s are all logically equivalent and can be written
139/// to the same column:
140/// * String, LargeString, StringView
141/// * Binary, LargeBinary, BinaryView
142///
143/// The writer can will also accept both native and dictionary encoded arrays if the dictionaries
144/// contain compatible values.
145/// ```
146/// # use std::sync::Arc;
147/// # use arrow_array::{DictionaryArray, LargeStringArray, RecordBatch, StringArray, UInt8Array};
148/// # use arrow_schema::{DataType, Field, Schema};
149/// # use parquet::arrow::arrow_writer::ArrowWriter;
150/// let record_batch1 = RecordBatch::try_new(
151///    Arc::new(Schema::new(vec![Field::new("col", DataType::LargeUtf8, false)])),
152///    vec![Arc::new(LargeStringArray::from_iter_values(vec!["a", "b"]))]
153///  )
154/// .unwrap();
155///
156/// let mut buffer = Vec::new();
157/// let mut writer = ArrowWriter::try_new(&mut buffer, record_batch1.schema(), None).unwrap();
158/// writer.write(&record_batch1).unwrap();
159///
160/// let record_batch2 = RecordBatch::try_new(
161///     Arc::new(Schema::new(vec![Field::new(
162///         "col",
163///         DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)),
164///          false,
165///     )])),
166///     vec![Arc::new(DictionaryArray::new(
167///          UInt8Array::from_iter_values(vec![0, 1]),
168///          Arc::new(StringArray::from_iter_values(vec!["b", "c"])),
169///      ))],
170///  )
171///  .unwrap();
172///  writer.write(&record_batch2).unwrap();
173///  writer.close();
174/// ```
175pub struct ArrowWriter<W: Write> {
176    /// Underlying Parquet writer
177    writer: SerializedFileWriter<W>,
178
179    /// The in-progress row group if any
180    in_progress: Option<ArrowRowGroupWriter>,
181
182    /// A copy of the Arrow schema.
183    ///
184    /// The schema is used to verify that each record batch written has the correct schema
185    arrow_schema: SchemaRef,
186
187    /// Creates new [`ArrowRowGroupWriter`] instances as required
188    row_group_writer_factory: ArrowRowGroupWriterFactory,
189
190    /// The length of arrays to write to each row group
191    max_row_group_size: usize,
192}
193
194impl<W: Write + Send> std::fmt::Debug for ArrowWriter<W> {
195    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
196        let buffered_memory = self.in_progress_size();
197        f.debug_struct("ArrowWriter")
198            .field("writer", &self.writer)
199            .field("in_progress_size", &format_args!("{buffered_memory} bytes"))
200            .field("in_progress_rows", &self.in_progress_rows())
201            .field("arrow_schema", &self.arrow_schema)
202            .field("max_row_group_size", &self.max_row_group_size)
203            .finish()
204    }
205}
206
207impl<W: Write + Send> ArrowWriter<W> {
208    /// Try to create a new Arrow writer
209    ///
210    /// The writer will fail if:
211    ///  * a `SerializedFileWriter` cannot be created from the ParquetWriter
212    ///  * the Arrow schema contains unsupported datatypes such as Unions
213    pub fn try_new(
214        writer: W,
215        arrow_schema: SchemaRef,
216        props: Option<WriterProperties>,
217    ) -> Result<Self> {
218        let options = ArrowWriterOptions::new().with_properties(props.unwrap_or_default());
219        Self::try_new_with_options(writer, arrow_schema, options)
220    }
221
222    /// Try to create a new Arrow writer with [`ArrowWriterOptions`].
223    ///
224    /// The writer will fail if:
225    ///  * a `SerializedFileWriter` cannot be created from the ParquetWriter
226    ///  * the Arrow schema contains unsupported datatypes such as Unions
227    pub fn try_new_with_options(
228        writer: W,
229        arrow_schema: SchemaRef,
230        options: ArrowWriterOptions,
231    ) -> Result<Self> {
232        let mut props = options.properties;
233
234        let schema = if let Some(parquet_schema) = options.schema_descr {
235            parquet_schema.clone()
236        } else {
237            let mut converter = ArrowSchemaConverter::new().with_coerce_types(props.coerce_types());
238            if let Some(schema_root) = &options.schema_root {
239                converter = converter.schema_root(schema_root);
240            }
241
242            converter.convert(&arrow_schema)?
243        };
244
245        if !options.skip_arrow_metadata {
246            // add serialized arrow schema
247            add_encoded_arrow_schema_to_metadata(&arrow_schema, &mut props);
248        }
249
250        let max_row_group_size = props.max_row_group_size();
251
252        let props_ptr = Arc::new(props);
253        let file_writer =
254            SerializedFileWriter::new(writer, schema.root_schema_ptr(), Arc::clone(&props_ptr))?;
255
256        let row_group_writer_factory =
257            ArrowRowGroupWriterFactory::new(&file_writer, arrow_schema.clone());
258
259        Ok(Self {
260            writer: file_writer,
261            in_progress: None,
262            arrow_schema,
263            row_group_writer_factory,
264            max_row_group_size,
265        })
266    }
267
268    /// Returns metadata for any flushed row groups
269    pub fn flushed_row_groups(&self) -> &[RowGroupMetaData] {
270        self.writer.flushed_row_groups()
271    }
272
273    /// Estimated memory usage, in bytes, of this `ArrowWriter`
274    ///
275    /// This estimate is formed bu summing the values of
276    /// [`ArrowColumnWriter::memory_size`] all in progress columns.
277    pub fn memory_size(&self) -> usize {
278        match &self.in_progress {
279            Some(in_progress) => in_progress.writers.iter().map(|x| x.memory_size()).sum(),
280            None => 0,
281        }
282    }
283
284    /// Anticipated encoded size of the in progress row group.
285    ///
286    /// This estimate the row group size after being completely encoded is,
287    /// formed by summing the values of
288    /// [`ArrowColumnWriter::get_estimated_total_bytes`] for all in progress
289    /// columns.
290    pub fn in_progress_size(&self) -> usize {
291        match &self.in_progress {
292            Some(in_progress) => in_progress
293                .writers
294                .iter()
295                .map(|x| x.get_estimated_total_bytes())
296                .sum(),
297            None => 0,
298        }
299    }
300
301    /// Returns the number of rows buffered in the in progress row group
302    pub fn in_progress_rows(&self) -> usize {
303        self.in_progress
304            .as_ref()
305            .map(|x| x.buffered_rows)
306            .unwrap_or_default()
307    }
308
309    /// Returns the number of bytes written by this instance
310    pub fn bytes_written(&self) -> usize {
311        self.writer.bytes_written()
312    }
313
314    /// Encodes the provided [`RecordBatch`]
315    ///
316    /// If this would cause the current row group to exceed [`WriterProperties::max_row_group_size`]
317    /// rows, the contents of `batch` will be written to one or more row groups such that all but
318    /// the final row group in the file contain [`WriterProperties::max_row_group_size`] rows.
319    ///
320    /// This will fail if the `batch`'s schema does not match the writer's schema.
321    pub fn write(&mut self, batch: &RecordBatch) -> Result<()> {
322        if batch.num_rows() == 0 {
323            return Ok(());
324        }
325
326        let in_progress = match &mut self.in_progress {
327            Some(in_progress) => in_progress,
328            x => x.insert(
329                self.row_group_writer_factory
330                    .create_row_group_writer(self.writer.flushed_row_groups().len())?,
331            ),
332        };
333
334        // If would exceed max_row_group_size, split batch
335        if in_progress.buffered_rows + batch.num_rows() > self.max_row_group_size {
336            let to_write = self.max_row_group_size - in_progress.buffered_rows;
337            let a = batch.slice(0, to_write);
338            let b = batch.slice(to_write, batch.num_rows() - to_write);
339            self.write(&a)?;
340            return self.write(&b);
341        }
342
343        in_progress.write(batch)?;
344
345        if in_progress.buffered_rows >= self.max_row_group_size {
346            self.flush()?
347        }
348        Ok(())
349    }
350
351    /// Writes the given buf bytes to the internal buffer.
352    ///
353    /// It's safe to use this method to write data to the underlying writer,
354    /// because it will ensure that the buffering and byte‐counting layers are used.
355    pub fn write_all(&mut self, buf: &[u8]) -> std::io::Result<()> {
356        self.writer.write_all(buf)
357    }
358
359    /// Flushes underlying writer
360    pub fn sync(&mut self) -> std::io::Result<()> {
361        self.writer.flush()
362    }
363
364    /// Flushes all buffered rows into a new row group
365    ///
366    /// Note the underlying writer is not flushed with this call.
367    /// If this is a desired behavior, please call [`ArrowWriter::sync`].
368    pub fn flush(&mut self) -> Result<()> {
369        let in_progress = match self.in_progress.take() {
370            Some(in_progress) => in_progress,
371            None => return Ok(()),
372        };
373
374        let mut row_group_writer = self.writer.next_row_group()?;
375        for chunk in in_progress.close()? {
376            chunk.append_to_row_group(&mut row_group_writer)?;
377        }
378        row_group_writer.close()?;
379        Ok(())
380    }
381
382    /// Additional [`KeyValue`] metadata to be written in addition to those from [`WriterProperties`]
383    ///
384    /// This method provide a way to append kv_metadata after write RecordBatch
385    pub fn append_key_value_metadata(&mut self, kv_metadata: KeyValue) {
386        self.writer.append_key_value_metadata(kv_metadata)
387    }
388
389    /// Returns a reference to the underlying writer.
390    pub fn inner(&self) -> &W {
391        self.writer.inner()
392    }
393
394    /// Returns a mutable reference to the underlying writer.
395    ///
396    /// **Warning**: if you write directly to this writer, you will skip
397    /// the `TrackedWrite` buffering and byte‐counting layers. That’ll cause
398    /// the file footer’s recorded offsets and sizes to diverge from reality,
399    /// resulting in an unreadable or corrupted Parquet file.
400    ///
401    /// If you want to write safely to the underlying writer, use [`Self::write_all`].
402    pub fn inner_mut(&mut self) -> &mut W {
403        self.writer.inner_mut()
404    }
405
406    /// Flushes any outstanding data and returns the underlying writer.
407    pub fn into_inner(mut self) -> Result<W> {
408        self.flush()?;
409        self.writer.into_inner()
410    }
411
412    /// Close and finalize the underlying Parquet writer
413    ///
414    /// Unlike [`Self::close`] this does not consume self
415    ///
416    /// Attempting to write after calling finish will result in an error
417    pub fn finish(&mut self) -> Result<ParquetMetaData> {
418        self.flush()?;
419        self.writer.finish()
420    }
421
422    /// Close and finalize the underlying Parquet writer
423    pub fn close(mut self) -> Result<ParquetMetaData> {
424        self.finish()
425    }
426
427    /// Create a new row group writer and return its column writers.
428    #[deprecated(
429        since = "56.2.0",
430        note = "Use `ArrowRowGroupWriterFactory` instead, see `ArrowColumnWriter` for an example"
431    )]
432    pub fn get_column_writers(&mut self) -> Result<Vec<ArrowColumnWriter>> {
433        self.flush()?;
434        let in_progress = self
435            .row_group_writer_factory
436            .create_row_group_writer(self.writer.flushed_row_groups().len())?;
437        Ok(in_progress.writers)
438    }
439
440    /// Append the given column chunks to the file as a new row group.
441    #[deprecated(
442        since = "56.2.0",
443        note = "Use `SerializedFileWriter` directly instead, see `ArrowColumnWriter` for an example"
444    )]
445    pub fn append_row_group(&mut self, chunks: Vec<ArrowColumnChunk>) -> Result<()> {
446        let mut row_group_writer = self.writer.next_row_group()?;
447        for chunk in chunks {
448            chunk.append_to_row_group(&mut row_group_writer)?;
449        }
450        row_group_writer.close()?;
451        Ok(())
452    }
453
454    /// Converts this writer into a lower-level [`SerializedFileWriter`] and [`ArrowRowGroupWriterFactory`].
455    ///
456    /// Flushes any outstanding data before returning.
457    ///
458    /// This can be useful to provide more control over how files are written, for example
459    /// to write columns in parallel. See the example on [`ArrowColumnWriter`].
460    pub fn into_serialized_writer(
461        mut self,
462    ) -> Result<(SerializedFileWriter<W>, ArrowRowGroupWriterFactory)> {
463        self.flush()?;
464        Ok((self.writer, self.row_group_writer_factory))
465    }
466}
467
468impl<W: Write + Send> RecordBatchWriter for ArrowWriter<W> {
469    fn write(&mut self, batch: &RecordBatch) -> Result<(), ArrowError> {
470        self.write(batch).map_err(|e| e.into())
471    }
472
473    fn close(self) -> std::result::Result<(), ArrowError> {
474        self.close()?;
475        Ok(())
476    }
477}
478
479/// Arrow-specific configuration settings for writing parquet files.
480///
481/// See [`ArrowWriter`] for how to configure the writer.
482#[derive(Debug, Clone, Default)]
483pub struct ArrowWriterOptions {
484    properties: WriterProperties,
485    skip_arrow_metadata: bool,
486    schema_root: Option<String>,
487    schema_descr: Option<SchemaDescriptor>,
488}
489
490impl ArrowWriterOptions {
491    /// Creates a new [`ArrowWriterOptions`] with the default settings.
492    pub fn new() -> Self {
493        Self::default()
494    }
495
496    /// Sets the [`WriterProperties`] for writing parquet files.
497    pub fn with_properties(self, properties: WriterProperties) -> Self {
498        Self { properties, ..self }
499    }
500
501    /// Skip encoding the embedded arrow metadata (defaults to `false`)
502    ///
503    /// Parquet files generated by the [`ArrowWriter`] contain embedded arrow schema
504    /// by default.
505    ///
506    /// Set `skip_arrow_metadata` to true, to skip encoding the embedded metadata.
507    pub fn with_skip_arrow_metadata(self, skip_arrow_metadata: bool) -> Self {
508        Self {
509            skip_arrow_metadata,
510            ..self
511        }
512    }
513
514    /// Set the name of the root parquet schema element (defaults to `"arrow_schema"`)
515    pub fn with_schema_root(self, schema_root: String) -> Self {
516        Self {
517            schema_root: Some(schema_root),
518            ..self
519        }
520    }
521
522    /// Explicitly specify the Parquet schema to be used
523    ///
524    /// If omitted (the default), the [`ArrowSchemaConverter`] is used to compute the
525    /// Parquet [`SchemaDescriptor`]. This may be used When the [`SchemaDescriptor`] is
526    /// already known or must be calculated using custom logic.
527    pub fn with_parquet_schema(self, schema_descr: SchemaDescriptor) -> Self {
528        Self {
529            schema_descr: Some(schema_descr),
530            ..self
531        }
532    }
533}
534
535/// A single column chunk produced by [`ArrowColumnWriter`]
536#[derive(Default)]
537struct ArrowColumnChunkData {
538    length: usize,
539    data: Vec<Bytes>,
540}
541
542impl Length for ArrowColumnChunkData {
543    fn len(&self) -> u64 {
544        self.length as _
545    }
546}
547
548impl ChunkReader for ArrowColumnChunkData {
549    type T = ArrowColumnChunkReader;
550
551    fn get_read(&self, start: u64) -> Result<Self::T> {
552        assert_eq!(start, 0); // Assume append_column writes all data in one-shot
553        Ok(ArrowColumnChunkReader(
554            self.data.clone().into_iter().peekable(),
555        ))
556    }
557
558    fn get_bytes(&self, _start: u64, _length: usize) -> Result<Bytes> {
559        unimplemented!()
560    }
561}
562
563/// A [`Read`] for [`ArrowColumnChunkData`]
564struct ArrowColumnChunkReader(Peekable<IntoIter<Bytes>>);
565
566impl Read for ArrowColumnChunkReader {
567    fn read(&mut self, out: &mut [u8]) -> std::io::Result<usize> {
568        let buffer = loop {
569            match self.0.peek_mut() {
570                Some(b) if b.is_empty() => {
571                    self.0.next();
572                    continue;
573                }
574                Some(b) => break b,
575                None => return Ok(0),
576            }
577        };
578
579        let len = buffer.len().min(out.len());
580        let b = buffer.split_to(len);
581        out[..len].copy_from_slice(&b);
582        Ok(len)
583    }
584}
585
586/// A shared [`ArrowColumnChunkData`]
587///
588/// This allows it to be owned by [`ArrowPageWriter`] whilst allowing access via
589/// [`ArrowRowGroupWriter`] on flush, without requiring self-referential borrows
590type SharedColumnChunk = Arc<Mutex<ArrowColumnChunkData>>;
591
592#[derive(Default)]
593struct ArrowPageWriter {
594    buffer: SharedColumnChunk,
595    #[cfg(feature = "encryption")]
596    page_encryptor: Option<PageEncryptor>,
597}
598
599impl ArrowPageWriter {
600    #[cfg(feature = "encryption")]
601    pub fn with_encryptor(mut self, page_encryptor: Option<PageEncryptor>) -> Self {
602        self.page_encryptor = page_encryptor;
603        self
604    }
605
606    #[cfg(feature = "encryption")]
607    fn page_encryptor_mut(&mut self) -> Option<&mut PageEncryptor> {
608        self.page_encryptor.as_mut()
609    }
610
611    #[cfg(not(feature = "encryption"))]
612    fn page_encryptor_mut(&mut self) -> Option<&mut PageEncryptor> {
613        None
614    }
615}
616
617impl PageWriter for ArrowPageWriter {
618    fn write_page(&mut self, page: CompressedPage) -> Result<PageWriteSpec> {
619        let page = match self.page_encryptor_mut() {
620            Some(page_encryptor) => page_encryptor.encrypt_compressed_page(page)?,
621            None => page,
622        };
623
624        let page_header = page.to_thrift_header()?;
625        let header = {
626            let mut header = Vec::with_capacity(1024);
627
628            match self.page_encryptor_mut() {
629                Some(page_encryptor) => {
630                    page_encryptor.encrypt_page_header(&page_header, &mut header)?;
631                    if page.compressed_page().is_data_page() {
632                        page_encryptor.increment_page();
633                    }
634                }
635                None => {
636                    let mut protocol = ThriftCompactOutputProtocol::new(&mut header);
637                    page_header.write_thrift(&mut protocol)?;
638                }
639            };
640
641            Bytes::from(header)
642        };
643
644        let mut buf = self.buffer.try_lock().unwrap();
645
646        let data = page.compressed_page().buffer().clone();
647        let compressed_size = data.len() + header.len();
648
649        let mut spec = PageWriteSpec::new();
650        spec.page_type = page.page_type();
651        spec.num_values = page.num_values();
652        spec.uncompressed_size = page.uncompressed_size() + header.len();
653        spec.offset = buf.length as u64;
654        spec.compressed_size = compressed_size;
655        spec.bytes_written = compressed_size as u64;
656
657        buf.length += compressed_size;
658        buf.data.push(header);
659        buf.data.push(data);
660
661        Ok(spec)
662    }
663
664    fn close(&mut self) -> Result<()> {
665        Ok(())
666    }
667}
668
669/// A leaf column that can be encoded by [`ArrowColumnWriter`]
670#[derive(Debug)]
671pub struct ArrowLeafColumn(ArrayLevels);
672
673/// Computes the [`ArrowLeafColumn`] for a potentially nested [`ArrayRef`]
674///
675/// This function can be used along with [`get_column_writers`] to encode
676/// individual columns in parallel. See example on [`ArrowColumnWriter`]
677pub fn compute_leaves(field: &Field, array: &ArrayRef) -> Result<Vec<ArrowLeafColumn>> {
678    let levels = calculate_array_levels(array, field)?;
679    Ok(levels.into_iter().map(ArrowLeafColumn).collect())
680}
681
682/// The data for a single column chunk, see [`ArrowColumnWriter`]
683pub struct ArrowColumnChunk {
684    data: ArrowColumnChunkData,
685    close: ColumnCloseResult,
686}
687
688impl std::fmt::Debug for ArrowColumnChunk {
689    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
690        f.debug_struct("ArrowColumnChunk")
691            .field("length", &self.data.length)
692            .finish_non_exhaustive()
693    }
694}
695
696impl ArrowColumnChunk {
697    /// Calls [`SerializedRowGroupWriter::append_column`] with this column's data
698    pub fn append_to_row_group<W: Write + Send>(
699        self,
700        writer: &mut SerializedRowGroupWriter<'_, W>,
701    ) -> Result<()> {
702        writer.append_column(&self.data, self.close)
703    }
704}
705
706/// Encodes [`ArrowLeafColumn`] to [`ArrowColumnChunk`]
707///
708/// `ArrowColumnWriter` instances can be created using an [`ArrowRowGroupWriterFactory`];
709///
710/// Note: This is a low-level interface for applications that require
711/// fine-grained control of encoding (e.g. encoding using multiple threads),
712/// see [`ArrowWriter`] for a higher-level interface
713///
714/// # Example: Encoding two Arrow Array's in Parallel
715/// ```
716/// // The arrow schema
717/// # use std::sync::Arc;
718/// # use arrow_array::*;
719/// # use arrow_schema::*;
720/// # use parquet::arrow::ArrowSchemaConverter;
721/// # use parquet::arrow::arrow_writer::{compute_leaves, ArrowColumnChunk, ArrowLeafColumn, ArrowRowGroupWriterFactory};
722/// # use parquet::file::properties::WriterProperties;
723/// # use parquet::file::writer::{SerializedFileWriter, SerializedRowGroupWriter};
724/// #
725/// let schema = Arc::new(Schema::new(vec![
726///     Field::new("i32", DataType::Int32, false),
727///     Field::new("f32", DataType::Float32, false),
728/// ]));
729///
730/// // Compute the parquet schema
731/// let props = Arc::new(WriterProperties::default());
732/// let parquet_schema = ArrowSchemaConverter::new()
733///   .with_coerce_types(props.coerce_types())
734///   .convert(&schema)
735///   .unwrap();
736///
737/// // Create parquet writer
738/// let root_schema = parquet_schema.root_schema_ptr();
739/// // write to memory in the example, but this could be a File
740/// let mut out = Vec::with_capacity(1024);
741/// let mut writer = SerializedFileWriter::new(&mut out, root_schema, props.clone())
742///   .unwrap();
743///
744/// // Create a factory for building Arrow column writers
745/// let row_group_factory = ArrowRowGroupWriterFactory::new(&writer, Arc::clone(&schema));
746/// // Create column writers for the 0th row group
747/// let col_writers = row_group_factory.create_column_writers(0).unwrap();
748///
749/// // Spawn a worker thread for each column
750/// //
751/// // Note: This is for demonstration purposes, a thread-pool e.g. rayon or tokio, would be better.
752/// // The `map` produces an iterator of type `tuple of (thread handle, send channel)`.
753/// let mut workers: Vec<_> = col_writers
754///     .into_iter()
755///     .map(|mut col_writer| {
756///         let (send, recv) = std::sync::mpsc::channel::<ArrowLeafColumn>();
757///         let handle = std::thread::spawn(move || {
758///             // receive Arrays to encode via the channel
759///             for col in recv {
760///                 col_writer.write(&col)?;
761///             }
762///             // once the input is complete, close the writer
763///             // to return the newly created ArrowColumnChunk
764///             col_writer.close()
765///         });
766///         (handle, send)
767///     })
768///     .collect();
769///
770/// // Start row group
771/// let mut row_group_writer: SerializedRowGroupWriter<'_, _> = writer
772///   .next_row_group()
773///   .unwrap();
774///
775/// // Create some example input columns to encode
776/// let to_write = vec![
777///     Arc::new(Int32Array::from_iter_values([1, 2, 3])) as _,
778///     Arc::new(Float32Array::from_iter_values([1., 45., -1.])) as _,
779/// ];
780///
781/// // Send the input columns to the workers
782/// let mut worker_iter = workers.iter_mut();
783/// for (arr, field) in to_write.iter().zip(&schema.fields) {
784///     for leaves in compute_leaves(field, arr).unwrap() {
785///         worker_iter.next().unwrap().1.send(leaves).unwrap();
786///     }
787/// }
788///
789/// // Wait for the workers to complete encoding, and append
790/// // the resulting column chunks to the row group (and the file)
791/// for (handle, send) in workers {
792///     drop(send); // Drop send side to signal termination
793///     // wait for the worker to send the completed chunk
794///     let chunk: ArrowColumnChunk = handle.join().unwrap().unwrap();
795///     chunk.append_to_row_group(&mut row_group_writer).unwrap();
796/// }
797/// // Close the row group which writes to the underlying file
798/// row_group_writer.close().unwrap();
799///
800/// let metadata = writer.close().unwrap();
801/// assert_eq!(metadata.file_metadata().num_rows(), 3);
802/// ```
803pub struct ArrowColumnWriter {
804    writer: ArrowColumnWriterImpl,
805    chunk: SharedColumnChunk,
806}
807
808impl std::fmt::Debug for ArrowColumnWriter {
809    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
810        f.debug_struct("ArrowColumnWriter").finish_non_exhaustive()
811    }
812}
813
814enum ArrowColumnWriterImpl {
815    ByteArray(GenericColumnWriter<'static, ByteArrayEncoder>),
816    Column(ColumnWriter<'static>),
817}
818
819impl ArrowColumnWriter {
820    /// Write an [`ArrowLeafColumn`]
821    pub fn write(&mut self, col: &ArrowLeafColumn) -> Result<()> {
822        match &mut self.writer {
823            ArrowColumnWriterImpl::Column(c) => {
824                let leaf = col.0.array();
825                match leaf.as_any_dictionary_opt() {
826                    Some(dictionary) => {
827                        let materialized =
828                            arrow_select::take::take(dictionary.values(), dictionary.keys(), None)?;
829                        write_leaf(c, &materialized, &col.0)?
830                    }
831                    None => write_leaf(c, leaf, &col.0)?,
832                };
833            }
834            ArrowColumnWriterImpl::ByteArray(c) => {
835                write_primitive(c, col.0.array().as_ref(), &col.0)?;
836            }
837        }
838        Ok(())
839    }
840
841    /// Close this column returning the written [`ArrowColumnChunk`]
842    pub fn close(self) -> Result<ArrowColumnChunk> {
843        let close = match self.writer {
844            ArrowColumnWriterImpl::ByteArray(c) => c.close()?,
845            ArrowColumnWriterImpl::Column(c) => c.close()?,
846        };
847        let chunk = Arc::try_unwrap(self.chunk).ok().unwrap();
848        let data = chunk.into_inner().unwrap();
849        Ok(ArrowColumnChunk { data, close })
850    }
851
852    /// Returns the estimated total memory usage by the writer.
853    ///
854    /// This  [`Self::get_estimated_total_bytes`] this is an estimate
855    /// of the current memory usage and not it's anticipated encoded size.
856    ///
857    /// This includes:
858    /// 1. Data buffered in encoded form
859    /// 2. Data buffered in un-encoded form (e.g. `usize` dictionary keys)
860    ///
861    /// This value should be greater than or equal to [`Self::get_estimated_total_bytes`]
862    pub fn memory_size(&self) -> usize {
863        match &self.writer {
864            ArrowColumnWriterImpl::ByteArray(c) => c.memory_size(),
865            ArrowColumnWriterImpl::Column(c) => c.memory_size(),
866        }
867    }
868
869    /// Returns the estimated total encoded bytes for this column writer.
870    ///
871    /// This includes:
872    /// 1. Data buffered in encoded form
873    /// 2. An estimate of how large the data buffered in un-encoded form would be once encoded
874    ///
875    /// This value should be less than or equal to [`Self::memory_size`]
876    pub fn get_estimated_total_bytes(&self) -> usize {
877        match &self.writer {
878            ArrowColumnWriterImpl::ByteArray(c) => c.get_estimated_total_bytes() as _,
879            ArrowColumnWriterImpl::Column(c) => c.get_estimated_total_bytes() as _,
880        }
881    }
882}
883
884/// Encodes [`RecordBatch`] to a parquet row group
885///
886/// Note: this structure is created by [`ArrowRowGroupWriterFactory`] internally used to
887/// create [`ArrowRowGroupWriter`]s, but it is not exposed publicly.
888///
889/// See the example on [`ArrowColumnWriter`] for how to encode columns in parallel
890#[derive(Debug)]
891struct ArrowRowGroupWriter {
892    writers: Vec<ArrowColumnWriter>,
893    schema: SchemaRef,
894    buffered_rows: usize,
895}
896
897impl ArrowRowGroupWriter {
898    fn new(writers: Vec<ArrowColumnWriter>, arrow: &SchemaRef) -> Self {
899        Self {
900            writers,
901            schema: arrow.clone(),
902            buffered_rows: 0,
903        }
904    }
905
906    fn write(&mut self, batch: &RecordBatch) -> Result<()> {
907        self.buffered_rows += batch.num_rows();
908        let mut writers = self.writers.iter_mut();
909        for (field, column) in self.schema.fields().iter().zip(batch.columns()) {
910            for leaf in compute_leaves(field.as_ref(), column)? {
911                writers.next().unwrap().write(&leaf)?
912            }
913        }
914        Ok(())
915    }
916
917    fn close(self) -> Result<Vec<ArrowColumnChunk>> {
918        self.writers
919            .into_iter()
920            .map(|writer| writer.close())
921            .collect()
922    }
923}
924
925/// Factory that creates new column writers for each row group in the Parquet file.
926///
927/// You can create this structure via an [`ArrowWriter::into_serialized_writer`].
928/// See the example on [`ArrowColumnWriter`] for how to encode columns in parallel
929#[derive(Debug)]
930pub struct ArrowRowGroupWriterFactory {
931    schema: SchemaDescPtr,
932    arrow_schema: SchemaRef,
933    props: WriterPropertiesPtr,
934    #[cfg(feature = "encryption")]
935    file_encryptor: Option<Arc<FileEncryptor>>,
936}
937
938impl ArrowRowGroupWriterFactory {
939    /// Create a new [`ArrowRowGroupWriterFactory`] for the provided file writer and Arrow schema
940    pub fn new<W: Write + Send>(
941        file_writer: &SerializedFileWriter<W>,
942        arrow_schema: SchemaRef,
943    ) -> Self {
944        let schema = Arc::clone(file_writer.schema_descr_ptr());
945        let props = Arc::clone(file_writer.properties());
946        Self {
947            schema,
948            arrow_schema,
949            props,
950            #[cfg(feature = "encryption")]
951            file_encryptor: file_writer.file_encryptor(),
952        }
953    }
954
955    fn create_row_group_writer(&self, row_group_index: usize) -> Result<ArrowRowGroupWriter> {
956        let writers = self.create_column_writers(row_group_index)?;
957        Ok(ArrowRowGroupWriter::new(writers, &self.arrow_schema))
958    }
959
960    /// Create column writers for a new row group, with the given row group index
961    pub fn create_column_writers(&self, row_group_index: usize) -> Result<Vec<ArrowColumnWriter>> {
962        let mut writers = Vec::with_capacity(self.arrow_schema.fields.len());
963        let mut leaves = self.schema.columns().iter();
964        let column_factory = self.column_writer_factory(row_group_index);
965        for field in &self.arrow_schema.fields {
966            column_factory.get_arrow_column_writer(
967                field.data_type(),
968                &self.props,
969                &mut leaves,
970                &mut writers,
971            )?;
972        }
973        Ok(writers)
974    }
975
976    #[cfg(feature = "encryption")]
977    fn column_writer_factory(&self, row_group_idx: usize) -> ArrowColumnWriterFactory {
978        ArrowColumnWriterFactory::new()
979            .with_file_encryptor(row_group_idx, self.file_encryptor.clone())
980    }
981
982    #[cfg(not(feature = "encryption"))]
983    fn column_writer_factory(&self, _row_group_idx: usize) -> ArrowColumnWriterFactory {
984        ArrowColumnWriterFactory::new()
985    }
986}
987
988/// Returns [`ArrowColumnWriter`]s for each column in a given schema
989#[deprecated(since = "57.0.0", note = "Use `ArrowRowGroupWriterFactory` instead")]
990pub fn get_column_writers(
991    parquet: &SchemaDescriptor,
992    props: &WriterPropertiesPtr,
993    arrow: &SchemaRef,
994) -> Result<Vec<ArrowColumnWriter>> {
995    let mut writers = Vec::with_capacity(arrow.fields.len());
996    let mut leaves = parquet.columns().iter();
997    let column_factory = ArrowColumnWriterFactory::new();
998    for field in &arrow.fields {
999        column_factory.get_arrow_column_writer(
1000            field.data_type(),
1001            props,
1002            &mut leaves,
1003            &mut writers,
1004        )?;
1005    }
1006    Ok(writers)
1007}
1008
1009/// Creates [`ArrowColumnWriter`] instances
1010struct ArrowColumnWriterFactory {
1011    #[cfg(feature = "encryption")]
1012    row_group_index: usize,
1013    #[cfg(feature = "encryption")]
1014    file_encryptor: Option<Arc<FileEncryptor>>,
1015}
1016
1017impl ArrowColumnWriterFactory {
1018    pub fn new() -> Self {
1019        Self {
1020            #[cfg(feature = "encryption")]
1021            row_group_index: 0,
1022            #[cfg(feature = "encryption")]
1023            file_encryptor: None,
1024        }
1025    }
1026
1027    #[cfg(feature = "encryption")]
1028    pub fn with_file_encryptor(
1029        mut self,
1030        row_group_index: usize,
1031        file_encryptor: Option<Arc<FileEncryptor>>,
1032    ) -> Self {
1033        self.row_group_index = row_group_index;
1034        self.file_encryptor = file_encryptor;
1035        self
1036    }
1037
1038    #[cfg(feature = "encryption")]
1039    fn create_page_writer(
1040        &self,
1041        column_descriptor: &ColumnDescPtr,
1042        column_index: usize,
1043    ) -> Result<Box<ArrowPageWriter>> {
1044        let column_path = column_descriptor.path().string();
1045        let page_encryptor = PageEncryptor::create_if_column_encrypted(
1046            &self.file_encryptor,
1047            self.row_group_index,
1048            column_index,
1049            &column_path,
1050        )?;
1051        Ok(Box::new(
1052            ArrowPageWriter::default().with_encryptor(page_encryptor),
1053        ))
1054    }
1055
1056    #[cfg(not(feature = "encryption"))]
1057    fn create_page_writer(
1058        &self,
1059        _column_descriptor: &ColumnDescPtr,
1060        _column_index: usize,
1061    ) -> Result<Box<ArrowPageWriter>> {
1062        Ok(Box::<ArrowPageWriter>::default())
1063    }
1064
1065    /// Gets an [`ArrowColumnWriter`] for the given `data_type`, appending the
1066    /// output ColumnDesc to `leaves` and the column writers to `out`
1067    fn get_arrow_column_writer(
1068        &self,
1069        data_type: &ArrowDataType,
1070        props: &WriterPropertiesPtr,
1071        leaves: &mut Iter<'_, ColumnDescPtr>,
1072        out: &mut Vec<ArrowColumnWriter>,
1073    ) -> Result<()> {
1074        // Instantiate writers for normal columns
1075        let col = |desc: &ColumnDescPtr| -> Result<ArrowColumnWriter> {
1076            let page_writer = self.create_page_writer(desc, out.len())?;
1077            let chunk = page_writer.buffer.clone();
1078            let writer = get_column_writer(desc.clone(), props.clone(), page_writer);
1079            Ok(ArrowColumnWriter {
1080                chunk,
1081                writer: ArrowColumnWriterImpl::Column(writer),
1082            })
1083        };
1084
1085        // Instantiate writers for byte arrays (e.g. Utf8,  Binary, etc)
1086        let bytes = |desc: &ColumnDescPtr| -> Result<ArrowColumnWriter> {
1087            let page_writer = self.create_page_writer(desc, out.len())?;
1088            let chunk = page_writer.buffer.clone();
1089            let writer = GenericColumnWriter::new(desc.clone(), props.clone(), page_writer);
1090            Ok(ArrowColumnWriter {
1091                chunk,
1092                writer: ArrowColumnWriterImpl::ByteArray(writer),
1093            })
1094        };
1095
1096        match data_type {
1097            _ if data_type.is_primitive() => out.push(col(leaves.next().unwrap())?),
1098            ArrowDataType::FixedSizeBinary(_) | ArrowDataType::Boolean | ArrowDataType::Null => {
1099                out.push(col(leaves.next().unwrap())?)
1100            }
1101            ArrowDataType::LargeBinary
1102            | ArrowDataType::Binary
1103            | ArrowDataType::Utf8
1104            | ArrowDataType::LargeUtf8
1105            | ArrowDataType::BinaryView
1106            | ArrowDataType::Utf8View => out.push(bytes(leaves.next().unwrap())?),
1107            ArrowDataType::List(f)
1108            | ArrowDataType::LargeList(f)
1109            | ArrowDataType::FixedSizeList(f, _) => {
1110                self.get_arrow_column_writer(f.data_type(), props, leaves, out)?
1111            }
1112            ArrowDataType::Struct(fields) => {
1113                for field in fields {
1114                    self.get_arrow_column_writer(field.data_type(), props, leaves, out)?
1115                }
1116            }
1117            ArrowDataType::Map(f, _) => match f.data_type() {
1118                ArrowDataType::Struct(f) => {
1119                    self.get_arrow_column_writer(f[0].data_type(), props, leaves, out)?;
1120                    self.get_arrow_column_writer(f[1].data_type(), props, leaves, out)?
1121                }
1122                _ => unreachable!("invalid map type"),
1123            },
1124            ArrowDataType::Dictionary(_, value_type) => match value_type.as_ref() {
1125                ArrowDataType::Utf8
1126                | ArrowDataType::LargeUtf8
1127                | ArrowDataType::Binary
1128                | ArrowDataType::LargeBinary => out.push(bytes(leaves.next().unwrap())?),
1129                ArrowDataType::Utf8View | ArrowDataType::BinaryView => {
1130                    out.push(bytes(leaves.next().unwrap())?)
1131                }
1132                ArrowDataType::FixedSizeBinary(_) => out.push(bytes(leaves.next().unwrap())?),
1133                _ => out.push(col(leaves.next().unwrap())?),
1134            },
1135            _ => {
1136                return Err(ParquetError::NYI(format!(
1137                    "Attempting to write an Arrow type {data_type} to parquet that is not yet implemented"
1138                )));
1139            }
1140        }
1141        Ok(())
1142    }
1143}
1144
1145fn write_leaf(
1146    writer: &mut ColumnWriter<'_>,
1147    column: &dyn arrow_array::Array,
1148    levels: &ArrayLevels,
1149) -> Result<usize> {
1150    let indices = levels.non_null_indices();
1151
1152    match writer {
1153        // Note: this should match the contents of arrow_to_parquet_type
1154        ColumnWriter::Int32ColumnWriter(typed) => {
1155            match column.data_type() {
1156                ArrowDataType::Null => {
1157                    let array = Int32Array::new_null(column.len());
1158                    write_primitive(typed, array.values(), levels)
1159                }
1160                ArrowDataType::Int8 => {
1161                    let array: Int32Array = column.as_primitive::<Int8Type>().unary(|x| x as i32);
1162                    write_primitive(typed, array.values(), levels)
1163                }
1164                ArrowDataType::Int16 => {
1165                    let array: Int32Array = column.as_primitive::<Int16Type>().unary(|x| x as i32);
1166                    write_primitive(typed, array.values(), levels)
1167                }
1168                ArrowDataType::Int32 => {
1169                    write_primitive(typed, column.as_primitive::<Int32Type>().values(), levels)
1170                }
1171                ArrowDataType::UInt8 => {
1172                    let array: Int32Array = column.as_primitive::<UInt8Type>().unary(|x| x as i32);
1173                    write_primitive(typed, array.values(), levels)
1174                }
1175                ArrowDataType::UInt16 => {
1176                    let array: Int32Array = column.as_primitive::<UInt16Type>().unary(|x| x as i32);
1177                    write_primitive(typed, array.values(), levels)
1178                }
1179                ArrowDataType::UInt32 => {
1180                    // follow C++ implementation and use overflow/reinterpret cast from  u32 to i32 which will map
1181                    // `(i32::MAX as u32)..u32::MAX` to `i32::MIN..0`
1182                    let array = column.as_primitive::<UInt32Type>();
1183                    write_primitive(typed, array.values().inner().typed_data(), levels)
1184                }
1185                ArrowDataType::Date32 => {
1186                    let array = column.as_primitive::<Date32Type>();
1187                    write_primitive(typed, array.values(), levels)
1188                }
1189                ArrowDataType::Time32(TimeUnit::Second) => {
1190                    let array = column.as_primitive::<Time32SecondType>();
1191                    write_primitive(typed, array.values(), levels)
1192                }
1193                ArrowDataType::Time32(TimeUnit::Millisecond) => {
1194                    let array = column.as_primitive::<Time32MillisecondType>();
1195                    write_primitive(typed, array.values(), levels)
1196                }
1197                ArrowDataType::Date64 => {
1198                    // If the column is a Date64, we truncate it
1199                    let array: Int32Array = column
1200                        .as_primitive::<Date64Type>()
1201                        .unary(|x| (x / 86_400_000) as _);
1202
1203                    write_primitive(typed, array.values(), levels)
1204                }
1205                ArrowDataType::Decimal32(_, _) => {
1206                    let array = column
1207                        .as_primitive::<Decimal32Type>()
1208                        .unary::<_, Int32Type>(|v| v);
1209                    write_primitive(typed, array.values(), levels)
1210                }
1211                ArrowDataType::Decimal64(_, _) => {
1212                    // use the int32 to represent the decimal with low precision
1213                    let array = column
1214                        .as_primitive::<Decimal64Type>()
1215                        .unary::<_, Int32Type>(|v| v as i32);
1216                    write_primitive(typed, array.values(), levels)
1217                }
1218                ArrowDataType::Decimal128(_, _) => {
1219                    // use the int32 to represent the decimal with low precision
1220                    let array = column
1221                        .as_primitive::<Decimal128Type>()
1222                        .unary::<_, Int32Type>(|v| v as i32);
1223                    write_primitive(typed, array.values(), levels)
1224                }
1225                ArrowDataType::Decimal256(_, _) => {
1226                    // use the int32 to represent the decimal with low precision
1227                    let array = column
1228                        .as_primitive::<Decimal256Type>()
1229                        .unary::<_, Int32Type>(|v| v.as_i128() as i32);
1230                    write_primitive(typed, array.values(), levels)
1231                }
1232                d => Err(ParquetError::General(format!("Cannot coerce {d} to I32"))),
1233            }
1234        }
1235        ColumnWriter::BoolColumnWriter(typed) => {
1236            let array = column.as_boolean();
1237            typed.write_batch(
1238                get_bool_array_slice(array, indices).as_slice(),
1239                levels.def_levels(),
1240                levels.rep_levels(),
1241            )
1242        }
1243        ColumnWriter::Int64ColumnWriter(typed) => {
1244            match column.data_type() {
1245                ArrowDataType::Date64 => {
1246                    let array = column
1247                        .as_primitive::<Date64Type>()
1248                        .reinterpret_cast::<Int64Type>();
1249
1250                    write_primitive(typed, array.values(), levels)
1251                }
1252                ArrowDataType::Int64 => {
1253                    let array = column.as_primitive::<Int64Type>();
1254                    write_primitive(typed, array.values(), levels)
1255                }
1256                ArrowDataType::UInt64 => {
1257                    let values = column.as_primitive::<UInt64Type>().values();
1258                    // follow C++ implementation and use overflow/reinterpret cast from  u64 to i64 which will map
1259                    // `(i64::MAX as u64)..u64::MAX` to `i64::MIN..0`
1260                    let array = values.inner().typed_data::<i64>();
1261                    write_primitive(typed, array, levels)
1262                }
1263                ArrowDataType::Time64(TimeUnit::Microsecond) => {
1264                    let array = column.as_primitive::<Time64MicrosecondType>();
1265                    write_primitive(typed, array.values(), levels)
1266                }
1267                ArrowDataType::Time64(TimeUnit::Nanosecond) => {
1268                    let array = column.as_primitive::<Time64NanosecondType>();
1269                    write_primitive(typed, array.values(), levels)
1270                }
1271                ArrowDataType::Timestamp(unit, _) => match unit {
1272                    TimeUnit::Second => {
1273                        let array = column.as_primitive::<TimestampSecondType>();
1274                        write_primitive(typed, array.values(), levels)
1275                    }
1276                    TimeUnit::Millisecond => {
1277                        let array = column.as_primitive::<TimestampMillisecondType>();
1278                        write_primitive(typed, array.values(), levels)
1279                    }
1280                    TimeUnit::Microsecond => {
1281                        let array = column.as_primitive::<TimestampMicrosecondType>();
1282                        write_primitive(typed, array.values(), levels)
1283                    }
1284                    TimeUnit::Nanosecond => {
1285                        let array = column.as_primitive::<TimestampNanosecondType>();
1286                        write_primitive(typed, array.values(), levels)
1287                    }
1288                },
1289                ArrowDataType::Duration(unit) => match unit {
1290                    TimeUnit::Second => {
1291                        let array = column.as_primitive::<DurationSecondType>();
1292                        write_primitive(typed, array.values(), levels)
1293                    }
1294                    TimeUnit::Millisecond => {
1295                        let array = column.as_primitive::<DurationMillisecondType>();
1296                        write_primitive(typed, array.values(), levels)
1297                    }
1298                    TimeUnit::Microsecond => {
1299                        let array = column.as_primitive::<DurationMicrosecondType>();
1300                        write_primitive(typed, array.values(), levels)
1301                    }
1302                    TimeUnit::Nanosecond => {
1303                        let array = column.as_primitive::<DurationNanosecondType>();
1304                        write_primitive(typed, array.values(), levels)
1305                    }
1306                },
1307                ArrowDataType::Decimal64(_, _) => {
1308                    let array = column
1309                        .as_primitive::<Decimal64Type>()
1310                        .reinterpret_cast::<Int64Type>();
1311                    write_primitive(typed, array.values(), levels)
1312                }
1313                ArrowDataType::Decimal128(_, _) => {
1314                    // use the int64 to represent the decimal with low precision
1315                    let array = column
1316                        .as_primitive::<Decimal128Type>()
1317                        .unary::<_, Int64Type>(|v| v as i64);
1318                    write_primitive(typed, array.values(), levels)
1319                }
1320                ArrowDataType::Decimal256(_, _) => {
1321                    // use the int64 to represent the decimal with low precision
1322                    let array = column
1323                        .as_primitive::<Decimal256Type>()
1324                        .unary::<_, Int64Type>(|v| v.as_i128() as i64);
1325                    write_primitive(typed, array.values(), levels)
1326                }
1327                d => Err(ParquetError::General(format!("Cannot coerce {d} to I64"))),
1328            }
1329        }
1330        ColumnWriter::Int96ColumnWriter(_typed) => {
1331            unreachable!("Currently unreachable because data type not supported")
1332        }
1333        ColumnWriter::FloatColumnWriter(typed) => {
1334            let array = column.as_primitive::<Float32Type>();
1335            write_primitive(typed, array.values(), levels)
1336        }
1337        ColumnWriter::DoubleColumnWriter(typed) => {
1338            let array = column.as_primitive::<Float64Type>();
1339            write_primitive(typed, array.values(), levels)
1340        }
1341        ColumnWriter::ByteArrayColumnWriter(_) => {
1342            unreachable!("should use ByteArrayWriter")
1343        }
1344        ColumnWriter::FixedLenByteArrayColumnWriter(typed) => {
1345            let bytes = match column.data_type() {
1346                ArrowDataType::Interval(interval_unit) => match interval_unit {
1347                    IntervalUnit::YearMonth => {
1348                        let array = column.as_primitive::<IntervalYearMonthType>();
1349                        get_interval_ym_array_slice(array, indices)
1350                    }
1351                    IntervalUnit::DayTime => {
1352                        let array = column.as_primitive::<IntervalDayTimeType>();
1353                        get_interval_dt_array_slice(array, indices)
1354                    }
1355                    _ => {
1356                        return Err(ParquetError::NYI(format!(
1357                            "Attempting to write an Arrow interval type {interval_unit:?} to parquet that is not yet implemented"
1358                        )));
1359                    }
1360                },
1361                ArrowDataType::FixedSizeBinary(_) => {
1362                    let array = column.as_fixed_size_binary();
1363                    get_fsb_array_slice(array, indices)
1364                }
1365                ArrowDataType::Decimal32(_, _) => {
1366                    let array = column.as_primitive::<Decimal32Type>();
1367                    get_decimal_32_array_slice(array, indices)
1368                }
1369                ArrowDataType::Decimal64(_, _) => {
1370                    let array = column.as_primitive::<Decimal64Type>();
1371                    get_decimal_64_array_slice(array, indices)
1372                }
1373                ArrowDataType::Decimal128(_, _) => {
1374                    let array = column.as_primitive::<Decimal128Type>();
1375                    get_decimal_128_array_slice(array, indices)
1376                }
1377                ArrowDataType::Decimal256(_, _) => {
1378                    let array = column.as_primitive::<Decimal256Type>();
1379                    get_decimal_256_array_slice(array, indices)
1380                }
1381                ArrowDataType::Float16 => {
1382                    let array = column.as_primitive::<Float16Type>();
1383                    get_float_16_array_slice(array, indices)
1384                }
1385                _ => {
1386                    return Err(ParquetError::NYI(
1387                        "Attempting to write an Arrow type that is not yet implemented".to_string(),
1388                    ));
1389                }
1390            };
1391            typed.write_batch(bytes.as_slice(), levels.def_levels(), levels.rep_levels())
1392        }
1393    }
1394}
1395
1396fn write_primitive<E: ColumnValueEncoder>(
1397    writer: &mut GenericColumnWriter<E>,
1398    values: &E::Values,
1399    levels: &ArrayLevels,
1400) -> Result<usize> {
1401    writer.write_batch_internal(
1402        values,
1403        Some(levels.non_null_indices()),
1404        levels.def_levels(),
1405        levels.rep_levels(),
1406        None,
1407        None,
1408        None,
1409    )
1410}
1411
1412fn get_bool_array_slice(array: &arrow_array::BooleanArray, indices: &[usize]) -> Vec<bool> {
1413    let mut values = Vec::with_capacity(indices.len());
1414    for i in indices {
1415        values.push(array.value(*i))
1416    }
1417    values
1418}
1419
1420/// Returns 12-byte values representing 3 values of months, days and milliseconds (4-bytes each).
1421/// An Arrow YearMonth interval only stores months, thus only the first 4 bytes are populated.
1422fn get_interval_ym_array_slice(
1423    array: &arrow_array::IntervalYearMonthArray,
1424    indices: &[usize],
1425) -> Vec<FixedLenByteArray> {
1426    let mut values = Vec::with_capacity(indices.len());
1427    for i in indices {
1428        let mut value = array.value(*i).to_le_bytes().to_vec();
1429        let mut suffix = vec![0; 8];
1430        value.append(&mut suffix);
1431        values.push(FixedLenByteArray::from(ByteArray::from(value)))
1432    }
1433    values
1434}
1435
1436/// Returns 12-byte values representing 3 values of months, days and milliseconds (4-bytes each).
1437/// An Arrow DayTime interval only stores days and millis, thus the first 4 bytes are not populated.
1438fn get_interval_dt_array_slice(
1439    array: &arrow_array::IntervalDayTimeArray,
1440    indices: &[usize],
1441) -> Vec<FixedLenByteArray> {
1442    let mut values = Vec::with_capacity(indices.len());
1443    for i in indices {
1444        let mut out = [0; 12];
1445        let value = array.value(*i);
1446        out[4..8].copy_from_slice(&value.days.to_le_bytes());
1447        out[8..12].copy_from_slice(&value.milliseconds.to_le_bytes());
1448        values.push(FixedLenByteArray::from(ByteArray::from(out.to_vec())));
1449    }
1450    values
1451}
1452
1453fn get_decimal_32_array_slice(
1454    array: &arrow_array::Decimal32Array,
1455    indices: &[usize],
1456) -> Vec<FixedLenByteArray> {
1457    let mut values = Vec::with_capacity(indices.len());
1458    let size = decimal_length_from_precision(array.precision());
1459    for i in indices {
1460        let as_be_bytes = array.value(*i).to_be_bytes();
1461        let resized_value = as_be_bytes[(4 - size)..].to_vec();
1462        values.push(FixedLenByteArray::from(ByteArray::from(resized_value)));
1463    }
1464    values
1465}
1466
1467fn get_decimal_64_array_slice(
1468    array: &arrow_array::Decimal64Array,
1469    indices: &[usize],
1470) -> Vec<FixedLenByteArray> {
1471    let mut values = Vec::with_capacity(indices.len());
1472    let size = decimal_length_from_precision(array.precision());
1473    for i in indices {
1474        let as_be_bytes = array.value(*i).to_be_bytes();
1475        let resized_value = as_be_bytes[(8 - size)..].to_vec();
1476        values.push(FixedLenByteArray::from(ByteArray::from(resized_value)));
1477    }
1478    values
1479}
1480
1481fn get_decimal_128_array_slice(
1482    array: &arrow_array::Decimal128Array,
1483    indices: &[usize],
1484) -> Vec<FixedLenByteArray> {
1485    let mut values = Vec::with_capacity(indices.len());
1486    let size = decimal_length_from_precision(array.precision());
1487    for i in indices {
1488        let as_be_bytes = array.value(*i).to_be_bytes();
1489        let resized_value = as_be_bytes[(16 - size)..].to_vec();
1490        values.push(FixedLenByteArray::from(ByteArray::from(resized_value)));
1491    }
1492    values
1493}
1494
1495fn get_decimal_256_array_slice(
1496    array: &arrow_array::Decimal256Array,
1497    indices: &[usize],
1498) -> Vec<FixedLenByteArray> {
1499    let mut values = Vec::with_capacity(indices.len());
1500    let size = decimal_length_from_precision(array.precision());
1501    for i in indices {
1502        let as_be_bytes = array.value(*i).to_be_bytes();
1503        let resized_value = as_be_bytes[(32 - size)..].to_vec();
1504        values.push(FixedLenByteArray::from(ByteArray::from(resized_value)));
1505    }
1506    values
1507}
1508
1509fn get_float_16_array_slice(
1510    array: &arrow_array::Float16Array,
1511    indices: &[usize],
1512) -> Vec<FixedLenByteArray> {
1513    let mut values = Vec::with_capacity(indices.len());
1514    for i in indices {
1515        let value = array.value(*i).to_le_bytes().to_vec();
1516        values.push(FixedLenByteArray::from(ByteArray::from(value)));
1517    }
1518    values
1519}
1520
1521fn get_fsb_array_slice(
1522    array: &arrow_array::FixedSizeBinaryArray,
1523    indices: &[usize],
1524) -> Vec<FixedLenByteArray> {
1525    let mut values = Vec::with_capacity(indices.len());
1526    for i in indices {
1527        let value = array.value(*i).to_vec();
1528        values.push(FixedLenByteArray::from(ByteArray::from(value)))
1529    }
1530    values
1531}
1532
1533#[cfg(test)]
1534mod tests {
1535    use super::*;
1536    use std::collections::HashMap;
1537
1538    use std::fs::File;
1539
1540    use crate::arrow::arrow_reader::{ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder};
1541    use crate::arrow::{ARROW_SCHEMA_META_KEY, PARQUET_FIELD_ID_META_KEY};
1542    use crate::column::page::{Page, PageReader};
1543    use crate::file::metadata::thrift::PageHeader;
1544    use crate::file::page_index::column_index::ColumnIndexMetaData;
1545    use crate::file::reader::SerializedPageReader;
1546    use crate::parquet_thrift::{ReadThrift, ThriftSliceInputProtocol};
1547    use crate::schema::types::ColumnPath;
1548    use arrow::datatypes::ToByteSlice;
1549    use arrow::datatypes::{DataType, Schema};
1550    use arrow::error::Result as ArrowResult;
1551    use arrow::util::data_gen::create_random_array;
1552    use arrow::util::pretty::pretty_format_batches;
1553    use arrow::{array::*, buffer::Buffer};
1554    use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, NullBuffer, OffsetBuffer, i256};
1555    use arrow_schema::Fields;
1556    use half::f16;
1557    use num_traits::{FromPrimitive, ToPrimitive};
1558    use tempfile::tempfile;
1559
1560    use crate::basic::Encoding;
1561    use crate::data_type::AsBytes;
1562    use crate::file::metadata::{ColumnChunkMetaData, ParquetMetaData, ParquetMetaDataReader};
1563    use crate::file::properties::{
1564        BloomFilterPosition, EnabledStatistics, ReaderProperties, WriterVersion,
1565    };
1566    use crate::file::serialized_reader::ReadOptionsBuilder;
1567    use crate::file::{
1568        reader::{FileReader, SerializedFileReader},
1569        statistics::Statistics,
1570    };
1571
1572    #[test]
1573    fn arrow_writer() {
1574        // define schema
1575        let schema = Schema::new(vec![
1576            Field::new("a", DataType::Int32, false),
1577            Field::new("b", DataType::Int32, true),
1578        ]);
1579
1580        // create some data
1581        let a = Int32Array::from(vec![1, 2, 3, 4, 5]);
1582        let b = Int32Array::from(vec![Some(1), None, None, Some(4), Some(5)]);
1583
1584        // build a record batch
1585        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)]).unwrap();
1586
1587        roundtrip(batch, Some(SMALL_SIZE / 2));
1588    }
1589
1590    fn get_bytes_after_close(schema: SchemaRef, expected_batch: &RecordBatch) -> Vec<u8> {
1591        let mut buffer = vec![];
1592
1593        let mut writer = ArrowWriter::try_new(&mut buffer, schema, None).unwrap();
1594        writer.write(expected_batch).unwrap();
1595        writer.close().unwrap();
1596
1597        buffer
1598    }
1599
1600    fn get_bytes_by_into_inner(schema: SchemaRef, expected_batch: &RecordBatch) -> Vec<u8> {
1601        let mut writer = ArrowWriter::try_new(Vec::new(), schema, None).unwrap();
1602        writer.write(expected_batch).unwrap();
1603        writer.into_inner().unwrap()
1604    }
1605
1606    #[test]
1607    fn roundtrip_bytes() {
1608        // define schema
1609        let schema = Arc::new(Schema::new(vec![
1610            Field::new("a", DataType::Int32, false),
1611            Field::new("b", DataType::Int32, true),
1612        ]));
1613
1614        // create some data
1615        let a = Int32Array::from(vec![1, 2, 3, 4, 5]);
1616        let b = Int32Array::from(vec![Some(1), None, None, Some(4), Some(5)]);
1617
1618        // build a record batch
1619        let expected_batch =
1620            RecordBatch::try_new(schema.clone(), vec![Arc::new(a), Arc::new(b)]).unwrap();
1621
1622        for buffer in [
1623            get_bytes_after_close(schema.clone(), &expected_batch),
1624            get_bytes_by_into_inner(schema, &expected_batch),
1625        ] {
1626            let cursor = Bytes::from(buffer);
1627            let mut record_batch_reader = ParquetRecordBatchReader::try_new(cursor, 1024).unwrap();
1628
1629            let actual_batch = record_batch_reader
1630                .next()
1631                .expect("No batch found")
1632                .expect("Unable to get batch");
1633
1634            assert_eq!(expected_batch.schema(), actual_batch.schema());
1635            assert_eq!(expected_batch.num_columns(), actual_batch.num_columns());
1636            assert_eq!(expected_batch.num_rows(), actual_batch.num_rows());
1637            for i in 0..expected_batch.num_columns() {
1638                let expected_data = expected_batch.column(i).to_data();
1639                let actual_data = actual_batch.column(i).to_data();
1640
1641                assert_eq!(expected_data, actual_data);
1642            }
1643        }
1644    }
1645
1646    #[test]
1647    fn arrow_writer_non_null() {
1648        // define schema
1649        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
1650
1651        // create some data
1652        let a = Int32Array::from(vec![1, 2, 3, 4, 5]);
1653
1654        // build a record batch
1655        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
1656
1657        roundtrip(batch, Some(SMALL_SIZE / 2));
1658    }
1659
1660    #[test]
1661    fn arrow_writer_list() {
1662        // define schema
1663        let schema = Schema::new(vec![Field::new(
1664            "a",
1665            DataType::List(Arc::new(Field::new_list_field(DataType::Int32, false))),
1666            true,
1667        )]);
1668
1669        // create some data
1670        let a_values = Int32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]);
1671
1672        // Construct a buffer for value offsets, for the nested array:
1673        //  [[1], [2, 3], null, [4, 5, 6], [7, 8, 9, 10]]
1674        let a_value_offsets = arrow::buffer::Buffer::from([0, 1, 3, 3, 6, 10].to_byte_slice());
1675
1676        // Construct a list array from the above two
1677        let a_list_data = ArrayData::builder(DataType::List(Arc::new(Field::new_list_field(
1678            DataType::Int32,
1679            false,
1680        ))))
1681        .len(5)
1682        .add_buffer(a_value_offsets)
1683        .add_child_data(a_values.into_data())
1684        .null_bit_buffer(Some(Buffer::from([0b00011011])))
1685        .build()
1686        .unwrap();
1687        let a = ListArray::from(a_list_data);
1688
1689        // build a record batch
1690        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
1691
1692        assert_eq!(batch.column(0).null_count(), 1);
1693
1694        // This test fails if the max row group size is less than the batch's length
1695        // see https://github.com/apache/arrow-rs/issues/518
1696        roundtrip(batch, None);
1697    }
1698
1699    #[test]
1700    fn arrow_writer_list_non_null() {
1701        // define schema
1702        let schema = Schema::new(vec![Field::new(
1703            "a",
1704            DataType::List(Arc::new(Field::new_list_field(DataType::Int32, false))),
1705            false,
1706        )]);
1707
1708        // create some data
1709        let a_values = Int32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]);
1710
1711        // Construct a buffer for value offsets, for the nested array:
1712        //  [[1], [2, 3], [], [4, 5, 6], [7, 8, 9, 10]]
1713        let a_value_offsets = arrow::buffer::Buffer::from([0, 1, 3, 3, 6, 10].to_byte_slice());
1714
1715        // Construct a list array from the above two
1716        let a_list_data = ArrayData::builder(DataType::List(Arc::new(Field::new_list_field(
1717            DataType::Int32,
1718            false,
1719        ))))
1720        .len(5)
1721        .add_buffer(a_value_offsets)
1722        .add_child_data(a_values.into_data())
1723        .build()
1724        .unwrap();
1725        let a = ListArray::from(a_list_data);
1726
1727        // build a record batch
1728        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
1729
1730        // This test fails if the max row group size is less than the batch's length
1731        // see https://github.com/apache/arrow-rs/issues/518
1732        assert_eq!(batch.column(0).null_count(), 0);
1733
1734        roundtrip(batch, None);
1735    }
1736
1737    #[test]
1738    fn arrow_writer_binary() {
1739        let string_field = Field::new("a", DataType::Utf8, false);
1740        let binary_field = Field::new("b", DataType::Binary, false);
1741        let schema = Schema::new(vec![string_field, binary_field]);
1742
1743        let raw_string_values = vec!["foo", "bar", "baz", "quux"];
1744        let raw_binary_values = [
1745            b"foo".to_vec(),
1746            b"bar".to_vec(),
1747            b"baz".to_vec(),
1748            b"quux".to_vec(),
1749        ];
1750        let raw_binary_value_refs = raw_binary_values
1751            .iter()
1752            .map(|x| x.as_slice())
1753            .collect::<Vec<_>>();
1754
1755        let string_values = StringArray::from(raw_string_values.clone());
1756        let binary_values = BinaryArray::from(raw_binary_value_refs);
1757        let batch = RecordBatch::try_new(
1758            Arc::new(schema),
1759            vec![Arc::new(string_values), Arc::new(binary_values)],
1760        )
1761        .unwrap();
1762
1763        roundtrip(batch, Some(SMALL_SIZE / 2));
1764    }
1765
1766    #[test]
1767    fn arrow_writer_binary_view() {
1768        let string_field = Field::new("a", DataType::Utf8View, false);
1769        let binary_field = Field::new("b", DataType::BinaryView, false);
1770        let nullable_string_field = Field::new("a", DataType::Utf8View, true);
1771        let schema = Schema::new(vec![string_field, binary_field, nullable_string_field]);
1772
1773        let raw_string_values = vec!["foo", "bar", "large payload over 12 bytes", "lulu"];
1774        let raw_binary_values = vec![
1775            b"foo".to_vec(),
1776            b"bar".to_vec(),
1777            b"large payload over 12 bytes".to_vec(),
1778            b"lulu".to_vec(),
1779        ];
1780        let nullable_string_values =
1781            vec![Some("foo"), None, Some("large payload over 12 bytes"), None];
1782
1783        let string_view_values = StringViewArray::from(raw_string_values);
1784        let binary_view_values = BinaryViewArray::from_iter_values(raw_binary_values);
1785        let nullable_string_view_values = StringViewArray::from(nullable_string_values);
1786        let batch = RecordBatch::try_new(
1787            Arc::new(schema),
1788            vec![
1789                Arc::new(string_view_values),
1790                Arc::new(binary_view_values),
1791                Arc::new(nullable_string_view_values),
1792            ],
1793        )
1794        .unwrap();
1795
1796        roundtrip(batch.clone(), Some(SMALL_SIZE / 2));
1797        roundtrip(batch, None);
1798    }
1799
1800    fn get_decimal_batch(precision: u8, scale: i8) -> RecordBatch {
1801        let decimal_field = Field::new("a", DataType::Decimal128(precision, scale), false);
1802        let schema = Schema::new(vec![decimal_field]);
1803
1804        let decimal_values = vec![10_000, 50_000, 0, -100]
1805            .into_iter()
1806            .map(Some)
1807            .collect::<Decimal128Array>()
1808            .with_precision_and_scale(precision, scale)
1809            .unwrap();
1810
1811        RecordBatch::try_new(Arc::new(schema), vec![Arc::new(decimal_values)]).unwrap()
1812    }
1813
1814    #[test]
1815    fn arrow_writer_decimal() {
1816        // int32 to store the decimal value
1817        let batch_int32_decimal = get_decimal_batch(5, 2);
1818        roundtrip(batch_int32_decimal, Some(SMALL_SIZE / 2));
1819        // int64 to store the decimal value
1820        let batch_int64_decimal = get_decimal_batch(12, 2);
1821        roundtrip(batch_int64_decimal, Some(SMALL_SIZE / 2));
1822        // fixed_length_byte_array to store the decimal value
1823        let batch_fixed_len_byte_array_decimal = get_decimal_batch(30, 2);
1824        roundtrip(batch_fixed_len_byte_array_decimal, Some(SMALL_SIZE / 2));
1825    }
1826
1827    #[test]
1828    fn arrow_writer_complex() {
1829        // define schema
1830        let struct_field_d = Arc::new(Field::new("d", DataType::Float64, true));
1831        let struct_field_f = Arc::new(Field::new("f", DataType::Float32, true));
1832        let struct_field_g = Arc::new(Field::new_list(
1833            "g",
1834            Field::new_list_field(DataType::Int16, true),
1835            false,
1836        ));
1837        let struct_field_h = Arc::new(Field::new_list(
1838            "h",
1839            Field::new_list_field(DataType::Int16, false),
1840            true,
1841        ));
1842        let struct_field_e = Arc::new(Field::new_struct(
1843            "e",
1844            vec![
1845                struct_field_f.clone(),
1846                struct_field_g.clone(),
1847                struct_field_h.clone(),
1848            ],
1849            false,
1850        ));
1851        let schema = Schema::new(vec![
1852            Field::new("a", DataType::Int32, false),
1853            Field::new("b", DataType::Int32, true),
1854            Field::new_struct(
1855                "c",
1856                vec![struct_field_d.clone(), struct_field_e.clone()],
1857                false,
1858            ),
1859        ]);
1860
1861        // create some data
1862        let a = Int32Array::from(vec![1, 2, 3, 4, 5]);
1863        let b = Int32Array::from(vec![Some(1), None, None, Some(4), Some(5)]);
1864        let d = Float64Array::from(vec![None, None, None, Some(1.0), None]);
1865        let f = Float32Array::from(vec![Some(0.0), None, Some(333.3), None, Some(5.25)]);
1866
1867        let g_value = Int16Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]);
1868
1869        // Construct a buffer for value offsets, for the nested array:
1870        //  [[1], [2, 3], [], [4, 5, 6], [7, 8, 9, 10]]
1871        let g_value_offsets = arrow::buffer::Buffer::from([0, 1, 3, 3, 6, 10].to_byte_slice());
1872
1873        // Construct a list array from the above two
1874        let g_list_data = ArrayData::builder(struct_field_g.data_type().clone())
1875            .len(5)
1876            .add_buffer(g_value_offsets.clone())
1877            .add_child_data(g_value.to_data())
1878            .build()
1879            .unwrap();
1880        let g = ListArray::from(g_list_data);
1881        // The difference between g and h is that h has a null bitmap
1882        let h_list_data = ArrayData::builder(struct_field_h.data_type().clone())
1883            .len(5)
1884            .add_buffer(g_value_offsets)
1885            .add_child_data(g_value.to_data())
1886            .null_bit_buffer(Some(Buffer::from([0b00011011])))
1887            .build()
1888            .unwrap();
1889        let h = ListArray::from(h_list_data);
1890
1891        let e = StructArray::from(vec![
1892            (struct_field_f, Arc::new(f) as ArrayRef),
1893            (struct_field_g, Arc::new(g) as ArrayRef),
1894            (struct_field_h, Arc::new(h) as ArrayRef),
1895        ]);
1896
1897        let c = StructArray::from(vec![
1898            (struct_field_d, Arc::new(d) as ArrayRef),
1899            (struct_field_e, Arc::new(e) as ArrayRef),
1900        ]);
1901
1902        // build a record batch
1903        let batch = RecordBatch::try_new(
1904            Arc::new(schema),
1905            vec![Arc::new(a), Arc::new(b), Arc::new(c)],
1906        )
1907        .unwrap();
1908
1909        roundtrip(batch.clone(), Some(SMALL_SIZE / 2));
1910        roundtrip(batch, Some(SMALL_SIZE / 3));
1911    }
1912
1913    #[test]
1914    fn arrow_writer_complex_mixed() {
1915        // This test was added while investigating https://github.com/apache/arrow-rs/issues/244.
1916        // It was subsequently fixed while investigating https://github.com/apache/arrow-rs/issues/245.
1917
1918        // define schema
1919        let offset_field = Arc::new(Field::new("offset", DataType::Int32, false));
1920        let partition_field = Arc::new(Field::new("partition", DataType::Int64, true));
1921        let topic_field = Arc::new(Field::new("topic", DataType::Utf8, true));
1922        let schema = Schema::new(vec![Field::new(
1923            "some_nested_object",
1924            DataType::Struct(Fields::from(vec![
1925                offset_field.clone(),
1926                partition_field.clone(),
1927                topic_field.clone(),
1928            ])),
1929            false,
1930        )]);
1931
1932        // create some data
1933        let offset = Int32Array::from(vec![1, 2, 3, 4, 5]);
1934        let partition = Int64Array::from(vec![Some(1), None, None, Some(4), Some(5)]);
1935        let topic = StringArray::from(vec![Some("A"), None, Some("A"), Some(""), None]);
1936
1937        let some_nested_object = StructArray::from(vec![
1938            (offset_field, Arc::new(offset) as ArrayRef),
1939            (partition_field, Arc::new(partition) as ArrayRef),
1940            (topic_field, Arc::new(topic) as ArrayRef),
1941        ]);
1942
1943        // build a record batch
1944        let batch =
1945            RecordBatch::try_new(Arc::new(schema), vec![Arc::new(some_nested_object)]).unwrap();
1946
1947        roundtrip(batch, Some(SMALL_SIZE / 2));
1948    }
1949
1950    #[test]
1951    fn arrow_writer_map() {
1952        // Note: we are using the JSON Arrow reader for brevity
1953        let json_content = r#"
1954        {"stocks":{"long": "$AAA", "short": "$BBB"}}
1955        {"stocks":{"long": null, "long": "$CCC", "short": null}}
1956        {"stocks":{"hedged": "$YYY", "long": null, "short": "$D"}}
1957        "#;
1958        let entries_struct_type = DataType::Struct(Fields::from(vec![
1959            Field::new("key", DataType::Utf8, false),
1960            Field::new("value", DataType::Utf8, true),
1961        ]));
1962        let stocks_field = Field::new(
1963            "stocks",
1964            DataType::Map(
1965                Arc::new(Field::new("entries", entries_struct_type, false)),
1966                false,
1967            ),
1968            true,
1969        );
1970        let schema = Arc::new(Schema::new(vec![stocks_field]));
1971        let builder = arrow::json::ReaderBuilder::new(schema).with_batch_size(64);
1972        let mut reader = builder.build(std::io::Cursor::new(json_content)).unwrap();
1973
1974        let batch = reader.next().unwrap().unwrap();
1975        roundtrip(batch, None);
1976    }
1977
1978    #[test]
1979    fn arrow_writer_2_level_struct() {
1980        // tests writing <struct<struct<primitive>>
1981        let field_c = Field::new("c", DataType::Int32, true);
1982        let field_b = Field::new("b", DataType::Struct(vec![field_c].into()), true);
1983        let type_a = DataType::Struct(vec![field_b.clone()].into());
1984        let field_a = Field::new("a", type_a, true);
1985        let schema = Schema::new(vec![field_a.clone()]);
1986
1987        // create data
1988        let c = Int32Array::from(vec![Some(1), None, Some(3), None, None, Some(6)]);
1989        let b_data = ArrayDataBuilder::new(field_b.data_type().clone())
1990            .len(6)
1991            .null_bit_buffer(Some(Buffer::from([0b00100111])))
1992            .add_child_data(c.into_data())
1993            .build()
1994            .unwrap();
1995        let b = StructArray::from(b_data);
1996        let a_data = ArrayDataBuilder::new(field_a.data_type().clone())
1997            .len(6)
1998            .null_bit_buffer(Some(Buffer::from([0b00101111])))
1999            .add_child_data(b.into_data())
2000            .build()
2001            .unwrap();
2002        let a = StructArray::from(a_data);
2003
2004        assert_eq!(a.null_count(), 1);
2005        assert_eq!(a.column(0).null_count(), 2);
2006
2007        // build a racord batch
2008        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
2009
2010        roundtrip(batch, Some(SMALL_SIZE / 2));
2011    }
2012
2013    #[test]
2014    fn arrow_writer_2_level_struct_non_null() {
2015        // tests writing <struct<struct<primitive>>
2016        let field_c = Field::new("c", DataType::Int32, false);
2017        let type_b = DataType::Struct(vec![field_c].into());
2018        let field_b = Field::new("b", type_b.clone(), false);
2019        let type_a = DataType::Struct(vec![field_b].into());
2020        let field_a = Field::new("a", type_a.clone(), false);
2021        let schema = Schema::new(vec![field_a]);
2022
2023        // create data
2024        let c = Int32Array::from(vec![1, 2, 3, 4, 5, 6]);
2025        let b_data = ArrayDataBuilder::new(type_b)
2026            .len(6)
2027            .add_child_data(c.into_data())
2028            .build()
2029            .unwrap();
2030        let b = StructArray::from(b_data);
2031        let a_data = ArrayDataBuilder::new(type_a)
2032            .len(6)
2033            .add_child_data(b.into_data())
2034            .build()
2035            .unwrap();
2036        let a = StructArray::from(a_data);
2037
2038        assert_eq!(a.null_count(), 0);
2039        assert_eq!(a.column(0).null_count(), 0);
2040
2041        // build a racord batch
2042        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
2043
2044        roundtrip(batch, Some(SMALL_SIZE / 2));
2045    }
2046
2047    #[test]
2048    fn arrow_writer_2_level_struct_mixed_null() {
2049        // tests writing <struct<struct<primitive>>
2050        let field_c = Field::new("c", DataType::Int32, false);
2051        let type_b = DataType::Struct(vec![field_c].into());
2052        let field_b = Field::new("b", type_b.clone(), true);
2053        let type_a = DataType::Struct(vec![field_b].into());
2054        let field_a = Field::new("a", type_a.clone(), false);
2055        let schema = Schema::new(vec![field_a]);
2056
2057        // create data
2058        let c = Int32Array::from(vec![1, 2, 3, 4, 5, 6]);
2059        let b_data = ArrayDataBuilder::new(type_b)
2060            .len(6)
2061            .null_bit_buffer(Some(Buffer::from([0b00100111])))
2062            .add_child_data(c.into_data())
2063            .build()
2064            .unwrap();
2065        let b = StructArray::from(b_data);
2066        // a intentionally has no null buffer, to test that this is handled correctly
2067        let a_data = ArrayDataBuilder::new(type_a)
2068            .len(6)
2069            .add_child_data(b.into_data())
2070            .build()
2071            .unwrap();
2072        let a = StructArray::from(a_data);
2073
2074        assert_eq!(a.null_count(), 0);
2075        assert_eq!(a.column(0).null_count(), 2);
2076
2077        // build a racord batch
2078        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
2079
2080        roundtrip(batch, Some(SMALL_SIZE / 2));
2081    }
2082
2083    #[test]
2084    fn arrow_writer_2_level_struct_mixed_null_2() {
2085        // tests writing <struct<struct<primitive>>, where the primitive columns are non-null.
2086        let field_c = Field::new("c", DataType::Int32, false);
2087        let field_d = Field::new("d", DataType::FixedSizeBinary(4), false);
2088        let field_e = Field::new(
2089            "e",
2090            DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
2091            false,
2092        );
2093
2094        let field_b = Field::new(
2095            "b",
2096            DataType::Struct(vec![field_c, field_d, field_e].into()),
2097            false,
2098        );
2099        let type_a = DataType::Struct(vec![field_b.clone()].into());
2100        let field_a = Field::new("a", type_a, true);
2101        let schema = Schema::new(vec![field_a.clone()]);
2102
2103        // create data
2104        let c = Int32Array::from_iter_values(0..6);
2105        let d = FixedSizeBinaryArray::try_from_iter(
2106            ["aaaa", "bbbb", "cccc", "dddd", "eeee", "ffff"].into_iter(),
2107        )
2108        .expect("four byte values");
2109        let e = Int32DictionaryArray::from_iter(["one", "two", "three", "four", "five", "one"]);
2110        let b_data = ArrayDataBuilder::new(field_b.data_type().clone())
2111            .len(6)
2112            .add_child_data(c.into_data())
2113            .add_child_data(d.into_data())
2114            .add_child_data(e.into_data())
2115            .build()
2116            .unwrap();
2117        let b = StructArray::from(b_data);
2118        let a_data = ArrayDataBuilder::new(field_a.data_type().clone())
2119            .len(6)
2120            .null_bit_buffer(Some(Buffer::from([0b00100101])))
2121            .add_child_data(b.into_data())
2122            .build()
2123            .unwrap();
2124        let a = StructArray::from(a_data);
2125
2126        assert_eq!(a.null_count(), 3);
2127        assert_eq!(a.column(0).null_count(), 0);
2128
2129        // build a record batch
2130        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
2131
2132        roundtrip(batch, Some(SMALL_SIZE / 2));
2133    }
2134
2135    #[test]
2136    fn test_fixed_size_binary_in_dict() {
2137        fn test_fixed_size_binary_in_dict_inner<K>()
2138        where
2139            K: ArrowDictionaryKeyType,
2140            K::Native: FromPrimitive + ToPrimitive + TryFrom<u8>,
2141            <<K as arrow_array::ArrowPrimitiveType>::Native as TryFrom<u8>>::Error: std::fmt::Debug,
2142        {
2143            let field = Field::new(
2144                "a",
2145                DataType::Dictionary(
2146                    Box::new(K::DATA_TYPE),
2147                    Box::new(DataType::FixedSizeBinary(4)),
2148                ),
2149                false,
2150            );
2151            let schema = Schema::new(vec![field]);
2152
2153            let keys: Vec<K::Native> = vec![
2154                K::Native::try_from(0u8).unwrap(),
2155                K::Native::try_from(0u8).unwrap(),
2156                K::Native::try_from(1u8).unwrap(),
2157            ];
2158            let keys = PrimitiveArray::<K>::from_iter_values(keys);
2159            let values = FixedSizeBinaryArray::try_from_iter(
2160                vec![vec![0, 0, 0, 0], vec![1, 1, 1, 1]].into_iter(),
2161            )
2162            .unwrap();
2163
2164            let data = DictionaryArray::<K>::new(keys, Arc::new(values));
2165            let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(data)]).unwrap();
2166            roundtrip(batch, None);
2167        }
2168
2169        test_fixed_size_binary_in_dict_inner::<UInt8Type>();
2170        test_fixed_size_binary_in_dict_inner::<UInt16Type>();
2171        test_fixed_size_binary_in_dict_inner::<UInt32Type>();
2172        test_fixed_size_binary_in_dict_inner::<UInt16Type>();
2173        test_fixed_size_binary_in_dict_inner::<Int8Type>();
2174        test_fixed_size_binary_in_dict_inner::<Int16Type>();
2175        test_fixed_size_binary_in_dict_inner::<Int32Type>();
2176        test_fixed_size_binary_in_dict_inner::<Int64Type>();
2177    }
2178
2179    #[test]
2180    fn test_empty_dict() {
2181        let struct_fields = Fields::from(vec![Field::new(
2182            "dict",
2183            DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
2184            false,
2185        )]);
2186
2187        let schema = Schema::new(vec![Field::new_struct(
2188            "struct",
2189            struct_fields.clone(),
2190            true,
2191        )]);
2192        let dictionary = Arc::new(DictionaryArray::new(
2193            Int32Array::new_null(5),
2194            Arc::new(StringArray::new_null(0)),
2195        ));
2196
2197        let s = StructArray::new(
2198            struct_fields,
2199            vec![dictionary],
2200            Some(NullBuffer::new_null(5)),
2201        );
2202
2203        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(s)]).unwrap();
2204        roundtrip(batch, None);
2205    }
2206    #[test]
2207    fn arrow_writer_page_size() {
2208        let schema = Arc::new(Schema::new(vec![Field::new("col", DataType::Utf8, false)]));
2209
2210        let mut builder = StringBuilder::with_capacity(100, 329 * 10_000);
2211
2212        // Generate an array of 10 unique 10 character string
2213        for i in 0..10 {
2214            let value = i
2215                .to_string()
2216                .repeat(10)
2217                .chars()
2218                .take(10)
2219                .collect::<String>();
2220
2221            builder.append_value(value);
2222        }
2223
2224        let array = Arc::new(builder.finish());
2225
2226        let batch = RecordBatch::try_new(schema, vec![array]).unwrap();
2227
2228        let file = tempfile::tempfile().unwrap();
2229
2230        // Set everything very low so we fallback to PLAIN encoding after the first row
2231        let props = WriterProperties::builder()
2232            .set_data_page_size_limit(1)
2233            .set_dictionary_page_size_limit(1)
2234            .set_write_batch_size(1)
2235            .build();
2236
2237        let mut writer =
2238            ArrowWriter::try_new(file.try_clone().unwrap(), batch.schema(), Some(props))
2239                .expect("Unable to write file");
2240        writer.write(&batch).unwrap();
2241        writer.close().unwrap();
2242
2243        let options = ReadOptionsBuilder::new().with_page_index().build();
2244        let reader =
2245            SerializedFileReader::new_with_options(file.try_clone().unwrap(), options).unwrap();
2246
2247        let column = reader.metadata().row_group(0).columns();
2248
2249        assert_eq!(column.len(), 1);
2250
2251        // We should write one row before falling back to PLAIN encoding so there should still be a
2252        // dictionary page.
2253        assert!(
2254            column[0].dictionary_page_offset().is_some(),
2255            "Expected a dictionary page"
2256        );
2257
2258        assert!(reader.metadata().offset_index().is_some());
2259        let offset_indexes = &reader.metadata().offset_index().unwrap()[0];
2260
2261        let page_locations = offset_indexes[0].page_locations.clone();
2262
2263        // We should fallback to PLAIN encoding after the first row and our max page size is 1 bytes
2264        // so we expect one dictionary encoded page and then a page per row thereafter.
2265        assert_eq!(
2266            page_locations.len(),
2267            10,
2268            "Expected 10 pages but got {page_locations:#?}"
2269        );
2270    }
2271
2272    #[test]
2273    fn arrow_writer_float_nans() {
2274        let f16_field = Field::new("a", DataType::Float16, false);
2275        let f32_field = Field::new("b", DataType::Float32, false);
2276        let f64_field = Field::new("c", DataType::Float64, false);
2277        let schema = Schema::new(vec![f16_field, f32_field, f64_field]);
2278
2279        let f16_values = (0..MEDIUM_SIZE)
2280            .map(|i| {
2281                Some(if i % 2 == 0 {
2282                    f16::NAN
2283                } else {
2284                    f16::from_f32(i as f32)
2285                })
2286            })
2287            .collect::<Float16Array>();
2288
2289        let f32_values = (0..MEDIUM_SIZE)
2290            .map(|i| Some(if i % 2 == 0 { f32::NAN } else { i as f32 }))
2291            .collect::<Float32Array>();
2292
2293        let f64_values = (0..MEDIUM_SIZE)
2294            .map(|i| Some(if i % 2 == 0 { f64::NAN } else { i as f64 }))
2295            .collect::<Float64Array>();
2296
2297        let batch = RecordBatch::try_new(
2298            Arc::new(schema),
2299            vec![
2300                Arc::new(f16_values),
2301                Arc::new(f32_values),
2302                Arc::new(f64_values),
2303            ],
2304        )
2305        .unwrap();
2306
2307        roundtrip(batch, None);
2308    }
2309
2310    const SMALL_SIZE: usize = 7;
2311    const MEDIUM_SIZE: usize = 63;
2312
2313    fn roundtrip(expected_batch: RecordBatch, max_row_group_size: Option<usize>) -> Vec<Bytes> {
2314        let mut files = vec![];
2315        for version in [WriterVersion::PARQUET_1_0, WriterVersion::PARQUET_2_0] {
2316            let mut props = WriterProperties::builder().set_writer_version(version);
2317
2318            if let Some(size) = max_row_group_size {
2319                props = props.set_max_row_group_size(size)
2320            }
2321
2322            let props = props.build();
2323            files.push(roundtrip_opts(&expected_batch, props))
2324        }
2325        files
2326    }
2327
2328    // Round trip the specified record batch with the specified writer properties,
2329    // to an in-memory file, and validate the arrays using the specified function.
2330    // Returns the in-memory file.
2331    fn roundtrip_opts_with_array_validation<F>(
2332        expected_batch: &RecordBatch,
2333        props: WriterProperties,
2334        validate: F,
2335    ) -> Bytes
2336    where
2337        F: Fn(&ArrayData, &ArrayData),
2338    {
2339        let mut file = vec![];
2340
2341        let mut writer = ArrowWriter::try_new(&mut file, expected_batch.schema(), Some(props))
2342            .expect("Unable to write file");
2343        writer.write(expected_batch).unwrap();
2344        writer.close().unwrap();
2345
2346        let file = Bytes::from(file);
2347        let mut record_batch_reader =
2348            ParquetRecordBatchReader::try_new(file.clone(), 1024).unwrap();
2349
2350        let actual_batch = record_batch_reader
2351            .next()
2352            .expect("No batch found")
2353            .expect("Unable to get batch");
2354
2355        assert_eq!(expected_batch.schema(), actual_batch.schema());
2356        assert_eq!(expected_batch.num_columns(), actual_batch.num_columns());
2357        assert_eq!(expected_batch.num_rows(), actual_batch.num_rows());
2358        for i in 0..expected_batch.num_columns() {
2359            let expected_data = expected_batch.column(i).to_data();
2360            let actual_data = actual_batch.column(i).to_data();
2361            validate(&expected_data, &actual_data);
2362        }
2363
2364        file
2365    }
2366
2367    fn roundtrip_opts(expected_batch: &RecordBatch, props: WriterProperties) -> Bytes {
2368        roundtrip_opts_with_array_validation(expected_batch, props, |a, b| {
2369            a.validate_full().expect("valid expected data");
2370            b.validate_full().expect("valid actual data");
2371            assert_eq!(a, b)
2372        })
2373    }
2374
2375    struct RoundTripOptions {
2376        values: ArrayRef,
2377        schema: SchemaRef,
2378        bloom_filter: bool,
2379        bloom_filter_position: BloomFilterPosition,
2380    }
2381
2382    impl RoundTripOptions {
2383        fn new(values: ArrayRef, nullable: bool) -> Self {
2384            let data_type = values.data_type().clone();
2385            let schema = Schema::new(vec![Field::new("col", data_type, nullable)]);
2386            Self {
2387                values,
2388                schema: Arc::new(schema),
2389                bloom_filter: false,
2390                bloom_filter_position: BloomFilterPosition::AfterRowGroup,
2391            }
2392        }
2393    }
2394
2395    fn one_column_roundtrip(values: ArrayRef, nullable: bool) -> Vec<Bytes> {
2396        one_column_roundtrip_with_options(RoundTripOptions::new(values, nullable))
2397    }
2398
2399    fn one_column_roundtrip_with_schema(values: ArrayRef, schema: SchemaRef) -> Vec<Bytes> {
2400        let mut options = RoundTripOptions::new(values, false);
2401        options.schema = schema;
2402        one_column_roundtrip_with_options(options)
2403    }
2404
2405    fn one_column_roundtrip_with_options(options: RoundTripOptions) -> Vec<Bytes> {
2406        let RoundTripOptions {
2407            values,
2408            schema,
2409            bloom_filter,
2410            bloom_filter_position,
2411        } = options;
2412
2413        let encodings = match values.data_type() {
2414            DataType::Utf8 | DataType::LargeUtf8 | DataType::Binary | DataType::LargeBinary => {
2415                vec![
2416                    Encoding::PLAIN,
2417                    Encoding::DELTA_BYTE_ARRAY,
2418                    Encoding::DELTA_LENGTH_BYTE_ARRAY,
2419                ]
2420            }
2421            DataType::Int64
2422            | DataType::Int32
2423            | DataType::Int16
2424            | DataType::Int8
2425            | DataType::UInt64
2426            | DataType::UInt32
2427            | DataType::UInt16
2428            | DataType::UInt8 => vec![
2429                Encoding::PLAIN,
2430                Encoding::DELTA_BINARY_PACKED,
2431                Encoding::BYTE_STREAM_SPLIT,
2432            ],
2433            DataType::Float32 | DataType::Float64 => {
2434                vec![Encoding::PLAIN, Encoding::BYTE_STREAM_SPLIT]
2435            }
2436            _ => vec![Encoding::PLAIN],
2437        };
2438
2439        let expected_batch = RecordBatch::try_new(schema, vec![values]).unwrap();
2440
2441        let row_group_sizes = [1024, SMALL_SIZE, SMALL_SIZE / 2, SMALL_SIZE / 2 + 1, 10];
2442
2443        let mut files = vec![];
2444        for dictionary_size in [0, 1, 1024] {
2445            for encoding in &encodings {
2446                for version in [WriterVersion::PARQUET_1_0, WriterVersion::PARQUET_2_0] {
2447                    for row_group_size in row_group_sizes {
2448                        let props = WriterProperties::builder()
2449                            .set_writer_version(version)
2450                            .set_max_row_group_size(row_group_size)
2451                            .set_dictionary_enabled(dictionary_size != 0)
2452                            .set_dictionary_page_size_limit(dictionary_size.max(1))
2453                            .set_encoding(*encoding)
2454                            .set_bloom_filter_enabled(bloom_filter)
2455                            .set_bloom_filter_position(bloom_filter_position)
2456                            .build();
2457
2458                        files.push(roundtrip_opts(&expected_batch, props))
2459                    }
2460                }
2461            }
2462        }
2463        files
2464    }
2465
2466    fn values_required<A, I>(iter: I) -> Vec<Bytes>
2467    where
2468        A: From<Vec<I::Item>> + Array + 'static,
2469        I: IntoIterator,
2470    {
2471        let raw_values: Vec<_> = iter.into_iter().collect();
2472        let values = Arc::new(A::from(raw_values));
2473        one_column_roundtrip(values, false)
2474    }
2475
2476    fn values_optional<A, I>(iter: I) -> Vec<Bytes>
2477    where
2478        A: From<Vec<Option<I::Item>>> + Array + 'static,
2479        I: IntoIterator,
2480    {
2481        let optional_raw_values: Vec<_> = iter
2482            .into_iter()
2483            .enumerate()
2484            .map(|(i, v)| if i % 2 == 0 { None } else { Some(v) })
2485            .collect();
2486        let optional_values = Arc::new(A::from(optional_raw_values));
2487        one_column_roundtrip(optional_values, true)
2488    }
2489
2490    fn required_and_optional<A, I>(iter: I)
2491    where
2492        A: From<Vec<I::Item>> + From<Vec<Option<I::Item>>> + Array + 'static,
2493        I: IntoIterator + Clone,
2494    {
2495        values_required::<A, I>(iter.clone());
2496        values_optional::<A, I>(iter);
2497    }
2498
2499    fn check_bloom_filter<T: AsBytes>(
2500        files: Vec<Bytes>,
2501        file_column: String,
2502        positive_values: Vec<T>,
2503        negative_values: Vec<T>,
2504    ) {
2505        files.into_iter().take(1).for_each(|file| {
2506            let file_reader = SerializedFileReader::new_with_options(
2507                file,
2508                ReadOptionsBuilder::new()
2509                    .with_reader_properties(
2510                        ReaderProperties::builder()
2511                            .set_read_bloom_filter(true)
2512                            .build(),
2513                    )
2514                    .build(),
2515            )
2516            .expect("Unable to open file as Parquet");
2517            let metadata = file_reader.metadata();
2518
2519            // Gets bloom filters from all row groups.
2520            let mut bloom_filters: Vec<_> = vec![];
2521            for (ri, row_group) in metadata.row_groups().iter().enumerate() {
2522                if let Some((column_index, _)) = row_group
2523                    .columns()
2524                    .iter()
2525                    .enumerate()
2526                    .find(|(_, column)| column.column_path().string() == file_column)
2527                {
2528                    let row_group_reader = file_reader
2529                        .get_row_group(ri)
2530                        .expect("Unable to read row group");
2531                    if let Some(sbbf) = row_group_reader.get_column_bloom_filter(column_index) {
2532                        bloom_filters.push(sbbf.clone());
2533                    } else {
2534                        panic!("No bloom filter for column named {file_column} found");
2535                    }
2536                } else {
2537                    panic!("No column named {file_column} found");
2538                }
2539            }
2540
2541            positive_values.iter().for_each(|value| {
2542                let found = bloom_filters.iter().find(|sbbf| sbbf.check(value));
2543                assert!(
2544                    found.is_some(),
2545                    "{}",
2546                    format!("Value {:?} should be in bloom filter", value.as_bytes())
2547                );
2548            });
2549
2550            negative_values.iter().for_each(|value| {
2551                let found = bloom_filters.iter().find(|sbbf| sbbf.check(value));
2552                assert!(
2553                    found.is_none(),
2554                    "{}",
2555                    format!("Value {:?} should not be in bloom filter", value.as_bytes())
2556                );
2557            });
2558        });
2559    }
2560
2561    #[test]
2562    fn all_null_primitive_single_column() {
2563        let values = Arc::new(Int32Array::from(vec![None; SMALL_SIZE]));
2564        one_column_roundtrip(values, true);
2565    }
2566    #[test]
2567    fn null_single_column() {
2568        let values = Arc::new(NullArray::new(SMALL_SIZE));
2569        one_column_roundtrip(values, true);
2570        // null arrays are always nullable, a test with non-nullable nulls fails
2571    }
2572
2573    #[test]
2574    fn bool_single_column() {
2575        required_and_optional::<BooleanArray, _>(
2576            [true, false].iter().cycle().copied().take(SMALL_SIZE),
2577        );
2578    }
2579
2580    #[test]
2581    fn bool_large_single_column() {
2582        let values = Arc::new(
2583            [None, Some(true), Some(false)]
2584                .iter()
2585                .cycle()
2586                .copied()
2587                .take(200_000)
2588                .collect::<BooleanArray>(),
2589        );
2590        let schema = Schema::new(vec![Field::new("col", values.data_type().clone(), true)]);
2591        let expected_batch = RecordBatch::try_new(Arc::new(schema), vec![values]).unwrap();
2592        let file = tempfile::tempfile().unwrap();
2593
2594        let mut writer =
2595            ArrowWriter::try_new(file.try_clone().unwrap(), expected_batch.schema(), None)
2596                .expect("Unable to write file");
2597        writer.write(&expected_batch).unwrap();
2598        writer.close().unwrap();
2599    }
2600
2601    #[test]
2602    fn check_page_offset_index_with_nan() {
2603        let values = Arc::new(Float64Array::from(vec![f64::NAN; 10]));
2604        let schema = Schema::new(vec![Field::new("col", DataType::Float64, true)]);
2605        let batch = RecordBatch::try_new(Arc::new(schema), vec![values]).unwrap();
2606
2607        let mut out = Vec::with_capacity(1024);
2608        let mut writer =
2609            ArrowWriter::try_new(&mut out, batch.schema(), None).expect("Unable to write file");
2610        writer.write(&batch).unwrap();
2611        let file_meta_data = writer.close().unwrap();
2612        for row_group in file_meta_data.row_groups() {
2613            for column in row_group.columns() {
2614                assert!(column.offset_index_offset().is_some());
2615                assert!(column.offset_index_length().is_some());
2616                assert!(column.column_index_offset().is_none());
2617                assert!(column.column_index_length().is_none());
2618            }
2619        }
2620    }
2621
2622    #[test]
2623    fn i8_single_column() {
2624        required_and_optional::<Int8Array, _>(0..SMALL_SIZE as i8);
2625    }
2626
2627    #[test]
2628    fn i16_single_column() {
2629        required_and_optional::<Int16Array, _>(0..SMALL_SIZE as i16);
2630    }
2631
2632    #[test]
2633    fn i32_single_column() {
2634        required_and_optional::<Int32Array, _>(0..SMALL_SIZE as i32);
2635    }
2636
2637    #[test]
2638    fn i64_single_column() {
2639        required_and_optional::<Int64Array, _>(0..SMALL_SIZE as i64);
2640    }
2641
2642    #[test]
2643    fn u8_single_column() {
2644        required_and_optional::<UInt8Array, _>(0..SMALL_SIZE as u8);
2645    }
2646
2647    #[test]
2648    fn u16_single_column() {
2649        required_and_optional::<UInt16Array, _>(0..SMALL_SIZE as u16);
2650    }
2651
2652    #[test]
2653    fn u32_single_column() {
2654        required_and_optional::<UInt32Array, _>(0..SMALL_SIZE as u32);
2655    }
2656
2657    #[test]
2658    fn u64_single_column() {
2659        required_and_optional::<UInt64Array, _>(0..SMALL_SIZE as u64);
2660    }
2661
2662    #[test]
2663    fn f32_single_column() {
2664        required_and_optional::<Float32Array, _>((0..SMALL_SIZE).map(|i| i as f32));
2665    }
2666
2667    #[test]
2668    fn f64_single_column() {
2669        required_and_optional::<Float64Array, _>((0..SMALL_SIZE).map(|i| i as f64));
2670    }
2671
2672    // The timestamp array types don't implement From<Vec<T>> because they need the timezone
2673    // argument, and they also doesn't support building from a Vec<Option<T>>, so call
2674    // one_column_roundtrip manually instead of calling required_and_optional for these tests.
2675
2676    #[test]
2677    fn timestamp_second_single_column() {
2678        let raw_values: Vec<_> = (0..SMALL_SIZE as i64).collect();
2679        let values = Arc::new(TimestampSecondArray::from(raw_values));
2680
2681        one_column_roundtrip(values, false);
2682    }
2683
2684    #[test]
2685    fn timestamp_millisecond_single_column() {
2686        let raw_values: Vec<_> = (0..SMALL_SIZE as i64).collect();
2687        let values = Arc::new(TimestampMillisecondArray::from(raw_values));
2688
2689        one_column_roundtrip(values, false);
2690    }
2691
2692    #[test]
2693    fn timestamp_microsecond_single_column() {
2694        let raw_values: Vec<_> = (0..SMALL_SIZE as i64).collect();
2695        let values = Arc::new(TimestampMicrosecondArray::from(raw_values));
2696
2697        one_column_roundtrip(values, false);
2698    }
2699
2700    #[test]
2701    fn timestamp_nanosecond_single_column() {
2702        let raw_values: Vec<_> = (0..SMALL_SIZE as i64).collect();
2703        let values = Arc::new(TimestampNanosecondArray::from(raw_values));
2704
2705        one_column_roundtrip(values, false);
2706    }
2707
2708    #[test]
2709    fn date32_single_column() {
2710        required_and_optional::<Date32Array, _>(0..SMALL_SIZE as i32);
2711    }
2712
2713    #[test]
2714    fn date64_single_column() {
2715        // Date64 must be a multiple of 86400000, see ARROW-10925
2716        required_and_optional::<Date64Array, _>(
2717            (0..(SMALL_SIZE as i64 * 86400000)).step_by(86400000),
2718        );
2719    }
2720
2721    #[test]
2722    fn time32_second_single_column() {
2723        required_and_optional::<Time32SecondArray, _>(0..SMALL_SIZE as i32);
2724    }
2725
2726    #[test]
2727    fn time32_millisecond_single_column() {
2728        required_and_optional::<Time32MillisecondArray, _>(0..SMALL_SIZE as i32);
2729    }
2730
2731    #[test]
2732    fn time64_microsecond_single_column() {
2733        required_and_optional::<Time64MicrosecondArray, _>(0..SMALL_SIZE as i64);
2734    }
2735
2736    #[test]
2737    fn time64_nanosecond_single_column() {
2738        required_and_optional::<Time64NanosecondArray, _>(0..SMALL_SIZE as i64);
2739    }
2740
2741    #[test]
2742    fn duration_second_single_column() {
2743        required_and_optional::<DurationSecondArray, _>(0..SMALL_SIZE as i64);
2744    }
2745
2746    #[test]
2747    fn duration_millisecond_single_column() {
2748        required_and_optional::<DurationMillisecondArray, _>(0..SMALL_SIZE as i64);
2749    }
2750
2751    #[test]
2752    fn duration_microsecond_single_column() {
2753        required_and_optional::<DurationMicrosecondArray, _>(0..SMALL_SIZE as i64);
2754    }
2755
2756    #[test]
2757    fn duration_nanosecond_single_column() {
2758        required_and_optional::<DurationNanosecondArray, _>(0..SMALL_SIZE as i64);
2759    }
2760
2761    #[test]
2762    fn interval_year_month_single_column() {
2763        required_and_optional::<IntervalYearMonthArray, _>(0..SMALL_SIZE as i32);
2764    }
2765
2766    #[test]
2767    fn interval_day_time_single_column() {
2768        required_and_optional::<IntervalDayTimeArray, _>(vec![
2769            IntervalDayTime::new(0, 1),
2770            IntervalDayTime::new(0, 3),
2771            IntervalDayTime::new(3, -2),
2772            IntervalDayTime::new(-200, 4),
2773        ]);
2774    }
2775
2776    #[test]
2777    #[should_panic(
2778        expected = "Attempting to write an Arrow interval type MonthDayNano to parquet that is not yet implemented"
2779    )]
2780    fn interval_month_day_nano_single_column() {
2781        required_and_optional::<IntervalMonthDayNanoArray, _>(vec![
2782            IntervalMonthDayNano::new(0, 1, 5),
2783            IntervalMonthDayNano::new(0, 3, 2),
2784            IntervalMonthDayNano::new(3, -2, -5),
2785            IntervalMonthDayNano::new(-200, 4, -1),
2786        ]);
2787    }
2788
2789    #[test]
2790    fn binary_single_column() {
2791        let one_vec: Vec<u8> = (0..SMALL_SIZE as u8).collect();
2792        let many_vecs: Vec<_> = std::iter::repeat_n(one_vec, SMALL_SIZE).collect();
2793        let many_vecs_iter = many_vecs.iter().map(|v| v.as_slice());
2794
2795        // BinaryArrays can't be built from Vec<Option<&str>>, so only call `values_required`
2796        values_required::<BinaryArray, _>(many_vecs_iter);
2797    }
2798
2799    #[test]
2800    fn binary_view_single_column() {
2801        let one_vec: Vec<u8> = (0..SMALL_SIZE as u8).collect();
2802        let many_vecs: Vec<_> = std::iter::repeat_n(one_vec, SMALL_SIZE).collect();
2803        let many_vecs_iter = many_vecs.iter().map(|v| v.as_slice());
2804
2805        // BinaryArrays can't be built from Vec<Option<&str>>, so only call `values_required`
2806        values_required::<BinaryViewArray, _>(many_vecs_iter);
2807    }
2808
2809    #[test]
2810    fn i32_column_bloom_filter_at_end() {
2811        let array = Arc::new(Int32Array::from_iter(0..SMALL_SIZE as i32));
2812        let mut options = RoundTripOptions::new(array, false);
2813        options.bloom_filter = true;
2814        options.bloom_filter_position = BloomFilterPosition::End;
2815
2816        let files = one_column_roundtrip_with_options(options);
2817        check_bloom_filter(
2818            files,
2819            "col".to_string(),
2820            (0..SMALL_SIZE as i32).collect(),
2821            (SMALL_SIZE as i32 + 1..SMALL_SIZE as i32 + 10).collect(),
2822        );
2823    }
2824
2825    #[test]
2826    fn i32_column_bloom_filter() {
2827        let array = Arc::new(Int32Array::from_iter(0..SMALL_SIZE as i32));
2828        let mut options = RoundTripOptions::new(array, false);
2829        options.bloom_filter = true;
2830
2831        let files = one_column_roundtrip_with_options(options);
2832        check_bloom_filter(
2833            files,
2834            "col".to_string(),
2835            (0..SMALL_SIZE as i32).collect(),
2836            (SMALL_SIZE as i32 + 1..SMALL_SIZE as i32 + 10).collect(),
2837        );
2838    }
2839
2840    #[test]
2841    fn binary_column_bloom_filter() {
2842        let one_vec: Vec<u8> = (0..SMALL_SIZE as u8).collect();
2843        let many_vecs: Vec<_> = std::iter::repeat_n(one_vec, SMALL_SIZE).collect();
2844        let many_vecs_iter = many_vecs.iter().map(|v| v.as_slice());
2845
2846        let array = Arc::new(BinaryArray::from_iter_values(many_vecs_iter));
2847        let mut options = RoundTripOptions::new(array, false);
2848        options.bloom_filter = true;
2849
2850        let files = one_column_roundtrip_with_options(options);
2851        check_bloom_filter(
2852            files,
2853            "col".to_string(),
2854            many_vecs,
2855            vec![vec![(SMALL_SIZE + 1) as u8]],
2856        );
2857    }
2858
2859    #[test]
2860    fn empty_string_null_column_bloom_filter() {
2861        let raw_values: Vec<_> = (0..SMALL_SIZE).map(|i| i.to_string()).collect();
2862        let raw_strs = raw_values.iter().map(|s| s.as_str());
2863
2864        let array = Arc::new(StringArray::from_iter_values(raw_strs));
2865        let mut options = RoundTripOptions::new(array, false);
2866        options.bloom_filter = true;
2867
2868        let files = one_column_roundtrip_with_options(options);
2869
2870        let optional_raw_values: Vec<_> = raw_values
2871            .iter()
2872            .enumerate()
2873            .filter_map(|(i, v)| if i % 2 == 0 { None } else { Some(v.as_str()) })
2874            .collect();
2875        // For null slots, empty string should not be in bloom filter.
2876        check_bloom_filter(files, "col".to_string(), optional_raw_values, vec![""]);
2877    }
2878
2879    #[test]
2880    fn large_binary_single_column() {
2881        let one_vec: Vec<u8> = (0..SMALL_SIZE as u8).collect();
2882        let many_vecs: Vec<_> = std::iter::repeat_n(one_vec, SMALL_SIZE).collect();
2883        let many_vecs_iter = many_vecs.iter().map(|v| v.as_slice());
2884
2885        // LargeBinaryArrays can't be built from Vec<Option<&str>>, so only call `values_required`
2886        values_required::<LargeBinaryArray, _>(many_vecs_iter);
2887    }
2888
2889    #[test]
2890    fn fixed_size_binary_single_column() {
2891        let mut builder = FixedSizeBinaryBuilder::new(4);
2892        builder.append_value(b"0123").unwrap();
2893        builder.append_null();
2894        builder.append_value(b"8910").unwrap();
2895        builder.append_value(b"1112").unwrap();
2896        let array = Arc::new(builder.finish());
2897
2898        one_column_roundtrip(array, true);
2899    }
2900
2901    #[test]
2902    fn string_single_column() {
2903        let raw_values: Vec<_> = (0..SMALL_SIZE).map(|i| i.to_string()).collect();
2904        let raw_strs = raw_values.iter().map(|s| s.as_str());
2905
2906        required_and_optional::<StringArray, _>(raw_strs);
2907    }
2908
2909    #[test]
2910    fn large_string_single_column() {
2911        let raw_values: Vec<_> = (0..SMALL_SIZE).map(|i| i.to_string()).collect();
2912        let raw_strs = raw_values.iter().map(|s| s.as_str());
2913
2914        required_and_optional::<LargeStringArray, _>(raw_strs);
2915    }
2916
2917    #[test]
2918    fn string_view_single_column() {
2919        let raw_values: Vec<_> = (0..SMALL_SIZE).map(|i| i.to_string()).collect();
2920        let raw_strs = raw_values.iter().map(|s| s.as_str());
2921
2922        required_and_optional::<StringViewArray, _>(raw_strs);
2923    }
2924
2925    #[test]
2926    fn null_list_single_column() {
2927        let null_field = Field::new_list_field(DataType::Null, true);
2928        let list_field = Field::new("emptylist", DataType::List(Arc::new(null_field)), true);
2929
2930        let schema = Schema::new(vec![list_field]);
2931
2932        // Build [[], null, [null, null]]
2933        let a_values = NullArray::new(2);
2934        let a_value_offsets = arrow::buffer::Buffer::from([0, 0, 0, 2].to_byte_slice());
2935        let a_list_data = ArrayData::builder(DataType::List(Arc::new(Field::new_list_field(
2936            DataType::Null,
2937            true,
2938        ))))
2939        .len(3)
2940        .add_buffer(a_value_offsets)
2941        .null_bit_buffer(Some(Buffer::from([0b00000101])))
2942        .add_child_data(a_values.into_data())
2943        .build()
2944        .unwrap();
2945
2946        let a = ListArray::from(a_list_data);
2947
2948        assert!(a.is_valid(0));
2949        assert!(!a.is_valid(1));
2950        assert!(a.is_valid(2));
2951
2952        assert_eq!(a.value(0).len(), 0);
2953        assert_eq!(a.value(2).len(), 2);
2954        assert_eq!(a.value(2).logical_nulls().unwrap().null_count(), 2);
2955
2956        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
2957        roundtrip(batch, None);
2958    }
2959
2960    #[test]
2961    fn list_single_column() {
2962        let a_values = Int32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]);
2963        let a_value_offsets = arrow::buffer::Buffer::from([0, 1, 3, 3, 6, 10].to_byte_slice());
2964        let a_list_data = ArrayData::builder(DataType::List(Arc::new(Field::new_list_field(
2965            DataType::Int32,
2966            false,
2967        ))))
2968        .len(5)
2969        .add_buffer(a_value_offsets)
2970        .null_bit_buffer(Some(Buffer::from([0b00011011])))
2971        .add_child_data(a_values.into_data())
2972        .build()
2973        .unwrap();
2974
2975        assert_eq!(a_list_data.null_count(), 1);
2976
2977        let a = ListArray::from(a_list_data);
2978        let values = Arc::new(a);
2979
2980        one_column_roundtrip(values, true);
2981    }
2982
2983    #[test]
2984    fn large_list_single_column() {
2985        let a_values = Int32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]);
2986        let a_value_offsets = arrow::buffer::Buffer::from([0i64, 1, 3, 3, 6, 10].to_byte_slice());
2987        let a_list_data = ArrayData::builder(DataType::LargeList(Arc::new(Field::new(
2988            "large_item",
2989            DataType::Int32,
2990            true,
2991        ))))
2992        .len(5)
2993        .add_buffer(a_value_offsets)
2994        .add_child_data(a_values.into_data())
2995        .null_bit_buffer(Some(Buffer::from([0b00011011])))
2996        .build()
2997        .unwrap();
2998
2999        // I think this setup is incorrect because this should pass
3000        assert_eq!(a_list_data.null_count(), 1);
3001
3002        let a = LargeListArray::from(a_list_data);
3003        let values = Arc::new(a);
3004
3005        one_column_roundtrip(values, true);
3006    }
3007
3008    #[test]
3009    fn list_nested_nulls() {
3010        use arrow::datatypes::Int32Type;
3011        let data = vec![
3012            Some(vec![Some(1)]),
3013            Some(vec![Some(2), Some(3)]),
3014            None,
3015            Some(vec![Some(4), Some(5), None]),
3016            Some(vec![None]),
3017            Some(vec![Some(6), Some(7)]),
3018        ];
3019
3020        let list = ListArray::from_iter_primitive::<Int32Type, _, _>(data.clone());
3021        one_column_roundtrip(Arc::new(list), true);
3022
3023        let list = LargeListArray::from_iter_primitive::<Int32Type, _, _>(data);
3024        one_column_roundtrip(Arc::new(list), true);
3025    }
3026
3027    #[test]
3028    fn struct_single_column() {
3029        let a_values = Int32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]);
3030        let struct_field_a = Arc::new(Field::new("f", DataType::Int32, false));
3031        let s = StructArray::from(vec![(struct_field_a, Arc::new(a_values) as ArrayRef)]);
3032
3033        let values = Arc::new(s);
3034        one_column_roundtrip(values, false);
3035    }
3036
3037    #[test]
3038    fn list_and_map_coerced_names() {
3039        // Create map and list with non-Parquet naming
3040        let list_field =
3041            Field::new_list("my_list", Field::new("item", DataType::Int32, false), false);
3042        let map_field = Field::new_map(
3043            "my_map",
3044            "entries",
3045            Field::new("keys", DataType::Int32, false),
3046            Field::new("values", DataType::Int32, true),
3047            false,
3048            true,
3049        );
3050
3051        let list_array = create_random_array(&list_field, 100, 0.0, 0.0).unwrap();
3052        let map_array = create_random_array(&map_field, 100, 0.0, 0.0).unwrap();
3053
3054        let arrow_schema = Arc::new(Schema::new(vec![list_field, map_field]));
3055
3056        // Write data to Parquet but coerce names to match spec
3057        let props = Some(WriterProperties::builder().set_coerce_types(true).build());
3058        let file = tempfile::tempfile().unwrap();
3059        let mut writer =
3060            ArrowWriter::try_new(file.try_clone().unwrap(), arrow_schema.clone(), props).unwrap();
3061
3062        let batch = RecordBatch::try_new(arrow_schema, vec![list_array, map_array]).unwrap();
3063        writer.write(&batch).unwrap();
3064        let file_metadata = writer.close().unwrap();
3065
3066        let schema = file_metadata.file_metadata().schema();
3067        // Coerced name of "item" should be "element"
3068        let list_field = &schema.get_fields()[0].get_fields()[0];
3069        assert_eq!(list_field.get_fields()[0].name(), "element");
3070
3071        let map_field = &schema.get_fields()[1].get_fields()[0];
3072        // Coerced name of "entries" should be "key_value"
3073        assert_eq!(map_field.name(), "key_value");
3074        // Coerced name of "keys" should be "key"
3075        assert_eq!(map_field.get_fields()[0].name(), "key");
3076        // Coerced name of "values" should be "value"
3077        assert_eq!(map_field.get_fields()[1].name(), "value");
3078
3079        // Double check schema after reading from the file
3080        let reader = SerializedFileReader::new(file).unwrap();
3081        let file_schema = reader.metadata().file_metadata().schema();
3082        let fields = file_schema.get_fields();
3083        let list_field = &fields[0].get_fields()[0];
3084        assert_eq!(list_field.get_fields()[0].name(), "element");
3085        let map_field = &fields[1].get_fields()[0];
3086        assert_eq!(map_field.name(), "key_value");
3087        assert_eq!(map_field.get_fields()[0].name(), "key");
3088        assert_eq!(map_field.get_fields()[1].name(), "value");
3089    }
3090
3091    #[test]
3092    fn fallback_flush_data_page() {
3093        //tests if the Fallback::flush_data_page clears all buffers correctly
3094        let raw_values: Vec<_> = (0..MEDIUM_SIZE).map(|i| i.to_string()).collect();
3095        let values = Arc::new(StringArray::from(raw_values));
3096        let encodings = vec![
3097            Encoding::DELTA_BYTE_ARRAY,
3098            Encoding::DELTA_LENGTH_BYTE_ARRAY,
3099        ];
3100        let data_type = values.data_type().clone();
3101        let schema = Arc::new(Schema::new(vec![Field::new("col", data_type, false)]));
3102        let expected_batch = RecordBatch::try_new(schema, vec![values]).unwrap();
3103
3104        let row_group_sizes = [1024, SMALL_SIZE, SMALL_SIZE / 2, SMALL_SIZE / 2 + 1, 10];
3105        let data_page_size_limit: usize = 32;
3106        let write_batch_size: usize = 16;
3107
3108        for encoding in &encodings {
3109            for row_group_size in row_group_sizes {
3110                let props = WriterProperties::builder()
3111                    .set_writer_version(WriterVersion::PARQUET_2_0)
3112                    .set_max_row_group_size(row_group_size)
3113                    .set_dictionary_enabled(false)
3114                    .set_encoding(*encoding)
3115                    .set_data_page_size_limit(data_page_size_limit)
3116                    .set_write_batch_size(write_batch_size)
3117                    .build();
3118
3119                roundtrip_opts_with_array_validation(&expected_batch, props, |a, b| {
3120                    let string_array_a = StringArray::from(a.clone());
3121                    let string_array_b = StringArray::from(b.clone());
3122                    let vec_a: Vec<&str> = string_array_a.iter().map(|v| v.unwrap()).collect();
3123                    let vec_b: Vec<&str> = string_array_b.iter().map(|v| v.unwrap()).collect();
3124                    assert_eq!(
3125                        vec_a, vec_b,
3126                        "failed for encoder: {encoding:?} and row_group_size: {row_group_size:?}"
3127                    );
3128                });
3129            }
3130        }
3131    }
3132
3133    #[test]
3134    fn arrow_writer_string_dictionary() {
3135        // define schema
3136        #[allow(deprecated)]
3137        let schema = Arc::new(Schema::new(vec![Field::new_dict(
3138            "dictionary",
3139            DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
3140            true,
3141            42,
3142            true,
3143        )]));
3144
3145        // create some data
3146        let d: Int32DictionaryArray = [Some("alpha"), None, Some("beta"), Some("alpha")]
3147            .iter()
3148            .copied()
3149            .collect();
3150
3151        // build a record batch
3152        one_column_roundtrip_with_schema(Arc::new(d), schema);
3153    }
3154
3155    #[test]
3156    fn arrow_writer_test_type_compatibility() {
3157        fn ensure_compatible_write<T1, T2>(array1: T1, array2: T2, expected_result: T1)
3158        where
3159            T1: Array + 'static,
3160            T2: Array + 'static,
3161        {
3162            let schema1 = Arc::new(Schema::new(vec![Field::new(
3163                "a",
3164                array1.data_type().clone(),
3165                false,
3166            )]));
3167
3168            let file = tempfile().unwrap();
3169            let mut writer =
3170                ArrowWriter::try_new(file.try_clone().unwrap(), schema1.clone(), None).unwrap();
3171
3172            let rb1 = RecordBatch::try_new(schema1.clone(), vec![Arc::new(array1)]).unwrap();
3173            writer.write(&rb1).unwrap();
3174
3175            let schema2 = Arc::new(Schema::new(vec![Field::new(
3176                "a",
3177                array2.data_type().clone(),
3178                false,
3179            )]));
3180            let rb2 = RecordBatch::try_new(schema2, vec![Arc::new(array2)]).unwrap();
3181            writer.write(&rb2).unwrap();
3182
3183            writer.close().unwrap();
3184
3185            let mut record_batch_reader =
3186                ParquetRecordBatchReader::try_new(file.try_clone().unwrap(), 1024).unwrap();
3187            let actual_batch = record_batch_reader.next().unwrap().unwrap();
3188
3189            let expected_batch =
3190                RecordBatch::try_new(schema1, vec![Arc::new(expected_result)]).unwrap();
3191            assert_eq!(actual_batch, expected_batch);
3192        }
3193
3194        // check compatibility between native and dictionaries
3195
3196        ensure_compatible_write(
3197            DictionaryArray::new(
3198                UInt8Array::from_iter_values(vec![0]),
3199                Arc::new(StringArray::from_iter_values(vec!["parquet"])),
3200            ),
3201            StringArray::from_iter_values(vec!["barquet"]),
3202            DictionaryArray::new(
3203                UInt8Array::from_iter_values(vec![0, 1]),
3204                Arc::new(StringArray::from_iter_values(vec!["parquet", "barquet"])),
3205            ),
3206        );
3207
3208        ensure_compatible_write(
3209            StringArray::from_iter_values(vec!["parquet"]),
3210            DictionaryArray::new(
3211                UInt8Array::from_iter_values(vec![0]),
3212                Arc::new(StringArray::from_iter_values(vec!["barquet"])),
3213            ),
3214            StringArray::from_iter_values(vec!["parquet", "barquet"]),
3215        );
3216
3217        // check compatibility between dictionaries with different key types
3218
3219        ensure_compatible_write(
3220            DictionaryArray::new(
3221                UInt8Array::from_iter_values(vec![0]),
3222                Arc::new(StringArray::from_iter_values(vec!["parquet"])),
3223            ),
3224            DictionaryArray::new(
3225                UInt16Array::from_iter_values(vec![0]),
3226                Arc::new(StringArray::from_iter_values(vec!["barquet"])),
3227            ),
3228            DictionaryArray::new(
3229                UInt8Array::from_iter_values(vec![0, 1]),
3230                Arc::new(StringArray::from_iter_values(vec!["parquet", "barquet"])),
3231            ),
3232        );
3233
3234        // check compatibility between dictionaries with different value types
3235        ensure_compatible_write(
3236            DictionaryArray::new(
3237                UInt8Array::from_iter_values(vec![0]),
3238                Arc::new(StringArray::from_iter_values(vec!["parquet"])),
3239            ),
3240            DictionaryArray::new(
3241                UInt8Array::from_iter_values(vec![0]),
3242                Arc::new(LargeStringArray::from_iter_values(vec!["barquet"])),
3243            ),
3244            DictionaryArray::new(
3245                UInt8Array::from_iter_values(vec![0, 1]),
3246                Arc::new(StringArray::from_iter_values(vec!["parquet", "barquet"])),
3247            ),
3248        );
3249
3250        // check compatibility between a dictionary and a native array with a different type
3251        ensure_compatible_write(
3252            DictionaryArray::new(
3253                UInt8Array::from_iter_values(vec![0]),
3254                Arc::new(StringArray::from_iter_values(vec!["parquet"])),
3255            ),
3256            LargeStringArray::from_iter_values(vec!["barquet"]),
3257            DictionaryArray::new(
3258                UInt8Array::from_iter_values(vec![0, 1]),
3259                Arc::new(StringArray::from_iter_values(vec!["parquet", "barquet"])),
3260            ),
3261        );
3262
3263        // check compatibility for string types
3264
3265        ensure_compatible_write(
3266            StringArray::from_iter_values(vec!["parquet"]),
3267            LargeStringArray::from_iter_values(vec!["barquet"]),
3268            StringArray::from_iter_values(vec!["parquet", "barquet"]),
3269        );
3270
3271        ensure_compatible_write(
3272            LargeStringArray::from_iter_values(vec!["parquet"]),
3273            StringArray::from_iter_values(vec!["barquet"]),
3274            LargeStringArray::from_iter_values(vec!["parquet", "barquet"]),
3275        );
3276
3277        ensure_compatible_write(
3278            StringArray::from_iter_values(vec!["parquet"]),
3279            StringViewArray::from_iter_values(vec!["barquet"]),
3280            StringArray::from_iter_values(vec!["parquet", "barquet"]),
3281        );
3282
3283        ensure_compatible_write(
3284            StringViewArray::from_iter_values(vec!["parquet"]),
3285            StringArray::from_iter_values(vec!["barquet"]),
3286            StringViewArray::from_iter_values(vec!["parquet", "barquet"]),
3287        );
3288
3289        ensure_compatible_write(
3290            LargeStringArray::from_iter_values(vec!["parquet"]),
3291            StringViewArray::from_iter_values(vec!["barquet"]),
3292            LargeStringArray::from_iter_values(vec!["parquet", "barquet"]),
3293        );
3294
3295        ensure_compatible_write(
3296            StringViewArray::from_iter_values(vec!["parquet"]),
3297            LargeStringArray::from_iter_values(vec!["barquet"]),
3298            StringViewArray::from_iter_values(vec!["parquet", "barquet"]),
3299        );
3300
3301        // check compatibility for binary types
3302
3303        ensure_compatible_write(
3304            BinaryArray::from_iter_values(vec![b"parquet"]),
3305            LargeBinaryArray::from_iter_values(vec![b"barquet"]),
3306            BinaryArray::from_iter_values(vec![b"parquet", b"barquet"]),
3307        );
3308
3309        ensure_compatible_write(
3310            LargeBinaryArray::from_iter_values(vec![b"parquet"]),
3311            BinaryArray::from_iter_values(vec![b"barquet"]),
3312            LargeBinaryArray::from_iter_values(vec![b"parquet", b"barquet"]),
3313        );
3314
3315        ensure_compatible_write(
3316            BinaryArray::from_iter_values(vec![b"parquet"]),
3317            BinaryViewArray::from_iter_values(vec![b"barquet"]),
3318            BinaryArray::from_iter_values(vec![b"parquet", b"barquet"]),
3319        );
3320
3321        ensure_compatible_write(
3322            BinaryViewArray::from_iter_values(vec![b"parquet"]),
3323            BinaryArray::from_iter_values(vec![b"barquet"]),
3324            BinaryViewArray::from_iter_values(vec![b"parquet", b"barquet"]),
3325        );
3326
3327        ensure_compatible_write(
3328            BinaryViewArray::from_iter_values(vec![b"parquet"]),
3329            LargeBinaryArray::from_iter_values(vec![b"barquet"]),
3330            BinaryViewArray::from_iter_values(vec![b"parquet", b"barquet"]),
3331        );
3332
3333        ensure_compatible_write(
3334            LargeBinaryArray::from_iter_values(vec![b"parquet"]),
3335            BinaryViewArray::from_iter_values(vec![b"barquet"]),
3336            LargeBinaryArray::from_iter_values(vec![b"parquet", b"barquet"]),
3337        );
3338
3339        // check compatibility for list types
3340
3341        let list_field_metadata = HashMap::from_iter(vec![(
3342            PARQUET_FIELD_ID_META_KEY.to_string(),
3343            "1".to_string(),
3344        )]);
3345        let list_field = Field::new_list_field(DataType::Int32, false);
3346
3347        let values1 = Arc::new(Int32Array::from(vec![0, 1, 2, 3, 4]));
3348        let offsets1 = OffsetBuffer::new(vec![0, 2, 5].into());
3349
3350        let values2 = Arc::new(Int32Array::from(vec![5, 6, 7, 8, 9]));
3351        let offsets2 = OffsetBuffer::new(vec![0, 3, 5].into());
3352
3353        let values_expected = Arc::new(Int32Array::from(vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9]));
3354        let offsets_expected = OffsetBuffer::new(vec![0, 2, 5, 8, 10].into());
3355
3356        ensure_compatible_write(
3357            // when the initial schema has the metadata ...
3358            ListArray::try_new(
3359                Arc::new(
3360                    list_field
3361                        .clone()
3362                        .with_metadata(list_field_metadata.clone()),
3363                ),
3364                offsets1,
3365                values1,
3366                None,
3367            )
3368            .unwrap(),
3369            // ... and some intermediate schema doesn't have the metadata
3370            ListArray::try_new(Arc::new(list_field.clone()), offsets2, values2, None).unwrap(),
3371            // ... the write will still go through, and the resulting schema will inherit the initial metadata
3372            ListArray::try_new(
3373                Arc::new(
3374                    list_field
3375                        .clone()
3376                        .with_metadata(list_field_metadata.clone()),
3377                ),
3378                offsets_expected,
3379                values_expected,
3380                None,
3381            )
3382            .unwrap(),
3383        );
3384    }
3385
3386    #[test]
3387    fn arrow_writer_primitive_dictionary() {
3388        // define schema
3389        #[allow(deprecated)]
3390        let schema = Arc::new(Schema::new(vec![Field::new_dict(
3391            "dictionary",
3392            DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::UInt32)),
3393            true,
3394            42,
3395            true,
3396        )]));
3397
3398        // create some data
3399        let mut builder = PrimitiveDictionaryBuilder::<UInt8Type, UInt32Type>::new();
3400        builder.append(12345678).unwrap();
3401        builder.append_null();
3402        builder.append(22345678).unwrap();
3403        builder.append(12345678).unwrap();
3404        let d = builder.finish();
3405
3406        one_column_roundtrip_with_schema(Arc::new(d), schema);
3407    }
3408
3409    #[test]
3410    fn arrow_writer_decimal32_dictionary() {
3411        let integers = vec![12345, 56789, 34567];
3412
3413        let keys = UInt8Array::from(vec![Some(0), None, Some(1), Some(2), Some(1)]);
3414
3415        let values = Decimal32Array::from(integers.clone())
3416            .with_precision_and_scale(5, 2)
3417            .unwrap();
3418
3419        let array = DictionaryArray::new(keys, Arc::new(values));
3420        one_column_roundtrip(Arc::new(array.clone()), true);
3421
3422        let values = Decimal32Array::from(integers)
3423            .with_precision_and_scale(9, 2)
3424            .unwrap();
3425
3426        let array = array.with_values(Arc::new(values));
3427        one_column_roundtrip(Arc::new(array), true);
3428    }
3429
3430    #[test]
3431    fn arrow_writer_decimal64_dictionary() {
3432        let integers = vec![12345, 56789, 34567];
3433
3434        let keys = UInt8Array::from(vec![Some(0), None, Some(1), Some(2), Some(1)]);
3435
3436        let values = Decimal64Array::from(integers.clone())
3437            .with_precision_and_scale(5, 2)
3438            .unwrap();
3439
3440        let array = DictionaryArray::new(keys, Arc::new(values));
3441        one_column_roundtrip(Arc::new(array.clone()), true);
3442
3443        let values = Decimal64Array::from(integers)
3444            .with_precision_and_scale(12, 2)
3445            .unwrap();
3446
3447        let array = array.with_values(Arc::new(values));
3448        one_column_roundtrip(Arc::new(array), true);
3449    }
3450
3451    #[test]
3452    fn arrow_writer_decimal128_dictionary() {
3453        let integers = vec![12345, 56789, 34567];
3454
3455        let keys = UInt8Array::from(vec![Some(0), None, Some(1), Some(2), Some(1)]);
3456
3457        let values = Decimal128Array::from(integers.clone())
3458            .with_precision_and_scale(5, 2)
3459            .unwrap();
3460
3461        let array = DictionaryArray::new(keys, Arc::new(values));
3462        one_column_roundtrip(Arc::new(array.clone()), true);
3463
3464        let values = Decimal128Array::from(integers)
3465            .with_precision_and_scale(12, 2)
3466            .unwrap();
3467
3468        let array = array.with_values(Arc::new(values));
3469        one_column_roundtrip(Arc::new(array), true);
3470    }
3471
3472    #[test]
3473    fn arrow_writer_decimal256_dictionary() {
3474        let integers = vec![
3475            i256::from_i128(12345),
3476            i256::from_i128(56789),
3477            i256::from_i128(34567),
3478        ];
3479
3480        let keys = UInt8Array::from(vec![Some(0), None, Some(1), Some(2), Some(1)]);
3481
3482        let values = Decimal256Array::from(integers.clone())
3483            .with_precision_and_scale(5, 2)
3484            .unwrap();
3485
3486        let array = DictionaryArray::new(keys, Arc::new(values));
3487        one_column_roundtrip(Arc::new(array.clone()), true);
3488
3489        let values = Decimal256Array::from(integers)
3490            .with_precision_and_scale(12, 2)
3491            .unwrap();
3492
3493        let array = array.with_values(Arc::new(values));
3494        one_column_roundtrip(Arc::new(array), true);
3495    }
3496
3497    #[test]
3498    fn arrow_writer_string_dictionary_unsigned_index() {
3499        // define schema
3500        #[allow(deprecated)]
3501        let schema = Arc::new(Schema::new(vec![Field::new_dict(
3502            "dictionary",
3503            DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)),
3504            true,
3505            42,
3506            true,
3507        )]));
3508
3509        // create some data
3510        let d: UInt8DictionaryArray = [Some("alpha"), None, Some("beta"), Some("alpha")]
3511            .iter()
3512            .copied()
3513            .collect();
3514
3515        one_column_roundtrip_with_schema(Arc::new(d), schema);
3516    }
3517
3518    #[test]
3519    fn u32_min_max() {
3520        // check values roundtrip through parquet
3521        let src = [
3522            u32::MIN,
3523            u32::MIN + 1,
3524            (i32::MAX as u32) - 1,
3525            i32::MAX as u32,
3526            (i32::MAX as u32) + 1,
3527            u32::MAX - 1,
3528            u32::MAX,
3529        ];
3530        let values = Arc::new(UInt32Array::from_iter_values(src.iter().cloned()));
3531        let files = one_column_roundtrip(values, false);
3532
3533        for file in files {
3534            // check statistics are valid
3535            let reader = SerializedFileReader::new(file).unwrap();
3536            let metadata = reader.metadata();
3537
3538            let mut row_offset = 0;
3539            for row_group in metadata.row_groups() {
3540                assert_eq!(row_group.num_columns(), 1);
3541                let column = row_group.column(0);
3542
3543                let num_values = column.num_values() as usize;
3544                let src_slice = &src[row_offset..row_offset + num_values];
3545                row_offset += column.num_values() as usize;
3546
3547                let stats = column.statistics().unwrap();
3548                if let Statistics::Int32(stats) = stats {
3549                    assert_eq!(
3550                        *stats.min_opt().unwrap() as u32,
3551                        *src_slice.iter().min().unwrap()
3552                    );
3553                    assert_eq!(
3554                        *stats.max_opt().unwrap() as u32,
3555                        *src_slice.iter().max().unwrap()
3556                    );
3557                } else {
3558                    panic!("Statistics::Int32 missing")
3559                }
3560            }
3561        }
3562    }
3563
3564    #[test]
3565    fn u64_min_max() {
3566        // check values roundtrip through parquet
3567        let src = [
3568            u64::MIN,
3569            u64::MIN + 1,
3570            (i64::MAX as u64) - 1,
3571            i64::MAX as u64,
3572            (i64::MAX as u64) + 1,
3573            u64::MAX - 1,
3574            u64::MAX,
3575        ];
3576        let values = Arc::new(UInt64Array::from_iter_values(src.iter().cloned()));
3577        let files = one_column_roundtrip(values, false);
3578
3579        for file in files {
3580            // check statistics are valid
3581            let reader = SerializedFileReader::new(file).unwrap();
3582            let metadata = reader.metadata();
3583
3584            let mut row_offset = 0;
3585            for row_group in metadata.row_groups() {
3586                assert_eq!(row_group.num_columns(), 1);
3587                let column = row_group.column(0);
3588
3589                let num_values = column.num_values() as usize;
3590                let src_slice = &src[row_offset..row_offset + num_values];
3591                row_offset += column.num_values() as usize;
3592
3593                let stats = column.statistics().unwrap();
3594                if let Statistics::Int64(stats) = stats {
3595                    assert_eq!(
3596                        *stats.min_opt().unwrap() as u64,
3597                        *src_slice.iter().min().unwrap()
3598                    );
3599                    assert_eq!(
3600                        *stats.max_opt().unwrap() as u64,
3601                        *src_slice.iter().max().unwrap()
3602                    );
3603                } else {
3604                    panic!("Statistics::Int64 missing")
3605                }
3606            }
3607        }
3608    }
3609
3610    #[test]
3611    fn statistics_null_counts_only_nulls() {
3612        // check that null-count statistics for "only NULL"-columns are correct
3613        let values = Arc::new(UInt64Array::from(vec![None, None]));
3614        let files = one_column_roundtrip(values, true);
3615
3616        for file in files {
3617            // check statistics are valid
3618            let reader = SerializedFileReader::new(file).unwrap();
3619            let metadata = reader.metadata();
3620            assert_eq!(metadata.num_row_groups(), 1);
3621            let row_group = metadata.row_group(0);
3622            assert_eq!(row_group.num_columns(), 1);
3623            let column = row_group.column(0);
3624            let stats = column.statistics().unwrap();
3625            assert_eq!(stats.null_count_opt(), Some(2));
3626        }
3627    }
3628
3629    #[test]
3630    fn test_list_of_struct_roundtrip() {
3631        // define schema
3632        let int_field = Field::new("a", DataType::Int32, true);
3633        let int_field2 = Field::new("b", DataType::Int32, true);
3634
3635        let int_builder = Int32Builder::with_capacity(10);
3636        let int_builder2 = Int32Builder::with_capacity(10);
3637
3638        let struct_builder = StructBuilder::new(
3639            vec![int_field, int_field2],
3640            vec![Box::new(int_builder), Box::new(int_builder2)],
3641        );
3642        let mut list_builder = ListBuilder::new(struct_builder);
3643
3644        // Construct the following array
3645        // [{a: 1, b: 2}], [], null, [null, null], [{a: null, b: 3}], [{a: 2, b: null}]
3646
3647        // [{a: 1, b: 2}]
3648        let values = list_builder.values();
3649        values
3650            .field_builder::<Int32Builder>(0)
3651            .unwrap()
3652            .append_value(1);
3653        values
3654            .field_builder::<Int32Builder>(1)
3655            .unwrap()
3656            .append_value(2);
3657        values.append(true);
3658        list_builder.append(true);
3659
3660        // []
3661        list_builder.append(true);
3662
3663        // null
3664        list_builder.append(false);
3665
3666        // [null, null]
3667        let values = list_builder.values();
3668        values
3669            .field_builder::<Int32Builder>(0)
3670            .unwrap()
3671            .append_null();
3672        values
3673            .field_builder::<Int32Builder>(1)
3674            .unwrap()
3675            .append_null();
3676        values.append(false);
3677        values
3678            .field_builder::<Int32Builder>(0)
3679            .unwrap()
3680            .append_null();
3681        values
3682            .field_builder::<Int32Builder>(1)
3683            .unwrap()
3684            .append_null();
3685        values.append(false);
3686        list_builder.append(true);
3687
3688        // [{a: null, b: 3}]
3689        let values = list_builder.values();
3690        values
3691            .field_builder::<Int32Builder>(0)
3692            .unwrap()
3693            .append_null();
3694        values
3695            .field_builder::<Int32Builder>(1)
3696            .unwrap()
3697            .append_value(3);
3698        values.append(true);
3699        list_builder.append(true);
3700
3701        // [{a: 2, b: null}]
3702        let values = list_builder.values();
3703        values
3704            .field_builder::<Int32Builder>(0)
3705            .unwrap()
3706            .append_value(2);
3707        values
3708            .field_builder::<Int32Builder>(1)
3709            .unwrap()
3710            .append_null();
3711        values.append(true);
3712        list_builder.append(true);
3713
3714        let array = Arc::new(list_builder.finish());
3715
3716        one_column_roundtrip(array, true);
3717    }
3718
3719    fn row_group_sizes(metadata: &ParquetMetaData) -> Vec<i64> {
3720        metadata.row_groups().iter().map(|x| x.num_rows()).collect()
3721    }
3722
3723    #[test]
3724    fn test_aggregates_records() {
3725        let arrays = [
3726            Int32Array::from((0..100).collect::<Vec<_>>()),
3727            Int32Array::from((0..50).collect::<Vec<_>>()),
3728            Int32Array::from((200..500).collect::<Vec<_>>()),
3729        ];
3730
3731        let schema = Arc::new(Schema::new(vec![Field::new(
3732            "int",
3733            ArrowDataType::Int32,
3734            false,
3735        )]));
3736
3737        let file = tempfile::tempfile().unwrap();
3738
3739        let props = WriterProperties::builder()
3740            .set_max_row_group_size(200)
3741            .build();
3742
3743        let mut writer =
3744            ArrowWriter::try_new(file.try_clone().unwrap(), schema.clone(), Some(props)).unwrap();
3745
3746        for array in arrays {
3747            let batch = RecordBatch::try_new(schema.clone(), vec![Arc::new(array)]).unwrap();
3748            writer.write(&batch).unwrap();
3749        }
3750
3751        writer.close().unwrap();
3752
3753        let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap();
3754        assert_eq!(&row_group_sizes(builder.metadata()), &[200, 200, 50]);
3755
3756        let batches = builder
3757            .with_batch_size(100)
3758            .build()
3759            .unwrap()
3760            .collect::<ArrowResult<Vec<_>>>()
3761            .unwrap();
3762
3763        assert_eq!(batches.len(), 5);
3764        assert!(batches.iter().all(|x| x.num_columns() == 1));
3765
3766        let batch_sizes: Vec<_> = batches.iter().map(|x| x.num_rows()).collect();
3767
3768        assert_eq!(&batch_sizes, &[100, 100, 100, 100, 50]);
3769
3770        let values: Vec<_> = batches
3771            .iter()
3772            .flat_map(|x| {
3773                x.column(0)
3774                    .as_any()
3775                    .downcast_ref::<Int32Array>()
3776                    .unwrap()
3777                    .values()
3778                    .iter()
3779                    .cloned()
3780            })
3781            .collect();
3782
3783        let expected_values: Vec<_> = [0..100, 0..50, 200..500].into_iter().flatten().collect();
3784        assert_eq!(&values, &expected_values)
3785    }
3786
3787    #[test]
3788    fn complex_aggregate() {
3789        // Tests aggregating nested data
3790        let field_a = Arc::new(Field::new("leaf_a", DataType::Int32, false));
3791        let field_b = Arc::new(Field::new("leaf_b", DataType::Int32, true));
3792        let struct_a = Arc::new(Field::new(
3793            "struct_a",
3794            DataType::Struct(vec![field_a.clone(), field_b.clone()].into()),
3795            true,
3796        ));
3797
3798        let list_a = Arc::new(Field::new("list", DataType::List(struct_a), true));
3799        let struct_b = Arc::new(Field::new(
3800            "struct_b",
3801            DataType::Struct(vec![list_a.clone()].into()),
3802            false,
3803        ));
3804
3805        let schema = Arc::new(Schema::new(vec![struct_b]));
3806
3807        // create nested data
3808        let field_a_array = Int32Array::from(vec![1, 2, 3, 4, 5, 6]);
3809        let field_b_array =
3810            Int32Array::from_iter(vec![Some(1), None, Some(2), None, None, Some(6)]);
3811
3812        let struct_a_array = StructArray::from(vec![
3813            (field_a.clone(), Arc::new(field_a_array) as ArrayRef),
3814            (field_b.clone(), Arc::new(field_b_array) as ArrayRef),
3815        ]);
3816
3817        let list_data = ArrayDataBuilder::new(list_a.data_type().clone())
3818            .len(5)
3819            .add_buffer(Buffer::from_iter(vec![
3820                0_i32, 1_i32, 1_i32, 3_i32, 3_i32, 5_i32,
3821            ]))
3822            .null_bit_buffer(Some(Buffer::from_iter(vec![
3823                true, false, true, false, true,
3824            ])))
3825            .child_data(vec![struct_a_array.into_data()])
3826            .build()
3827            .unwrap();
3828
3829        let list_a_array = Arc::new(ListArray::from(list_data)) as ArrayRef;
3830        let struct_b_array = StructArray::from(vec![(list_a.clone(), list_a_array)]);
3831
3832        let batch1 =
3833            RecordBatch::try_from_iter(vec![("struct_b", Arc::new(struct_b_array) as ArrayRef)])
3834                .unwrap();
3835
3836        let field_a_array = Int32Array::from(vec![6, 7, 8, 9, 10]);
3837        let field_b_array = Int32Array::from_iter(vec![None, None, None, Some(1), None]);
3838
3839        let struct_a_array = StructArray::from(vec![
3840            (field_a, Arc::new(field_a_array) as ArrayRef),
3841            (field_b, Arc::new(field_b_array) as ArrayRef),
3842        ]);
3843
3844        let list_data = ArrayDataBuilder::new(list_a.data_type().clone())
3845            .len(2)
3846            .add_buffer(Buffer::from_iter(vec![0_i32, 4_i32, 5_i32]))
3847            .child_data(vec![struct_a_array.into_data()])
3848            .build()
3849            .unwrap();
3850
3851        let list_a_array = Arc::new(ListArray::from(list_data)) as ArrayRef;
3852        let struct_b_array = StructArray::from(vec![(list_a, list_a_array)]);
3853
3854        let batch2 =
3855            RecordBatch::try_from_iter(vec![("struct_b", Arc::new(struct_b_array) as ArrayRef)])
3856                .unwrap();
3857
3858        let batches = &[batch1, batch2];
3859
3860        // Verify data is as expected
3861
3862        let expected = r#"
3863            +-------------------------------------------------------------------------------------------------------+
3864            | struct_b                                                                                              |
3865            +-------------------------------------------------------------------------------------------------------+
3866            | {list: [{leaf_a: 1, leaf_b: 1}]}                                                                      |
3867            | {list: }                                                                                              |
3868            | {list: [{leaf_a: 2, leaf_b: }, {leaf_a: 3, leaf_b: 2}]}                                               |
3869            | {list: }                                                                                              |
3870            | {list: [{leaf_a: 4, leaf_b: }, {leaf_a: 5, leaf_b: }]}                                                |
3871            | {list: [{leaf_a: 6, leaf_b: }, {leaf_a: 7, leaf_b: }, {leaf_a: 8, leaf_b: }, {leaf_a: 9, leaf_b: 1}]} |
3872            | {list: [{leaf_a: 10, leaf_b: }]}                                                                      |
3873            +-------------------------------------------------------------------------------------------------------+
3874        "#.trim().split('\n').map(|x| x.trim()).collect::<Vec<_>>().join("\n");
3875
3876        let actual = pretty_format_batches(batches).unwrap().to_string();
3877        assert_eq!(actual, expected);
3878
3879        // Write data
3880        let file = tempfile::tempfile().unwrap();
3881        let props = WriterProperties::builder()
3882            .set_max_row_group_size(6)
3883            .build();
3884
3885        let mut writer =
3886            ArrowWriter::try_new(file.try_clone().unwrap(), schema, Some(props)).unwrap();
3887
3888        for batch in batches {
3889            writer.write(batch).unwrap();
3890        }
3891        writer.close().unwrap();
3892
3893        // Read Data
3894        // Should have written entire first batch and first row of second to the first row group
3895        // leaving a single row in the second row group
3896
3897        let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap();
3898        assert_eq!(&row_group_sizes(builder.metadata()), &[6, 1]);
3899
3900        let batches = builder
3901            .with_batch_size(2)
3902            .build()
3903            .unwrap()
3904            .collect::<ArrowResult<Vec<_>>>()
3905            .unwrap();
3906
3907        assert_eq!(batches.len(), 4);
3908        let batch_counts: Vec<_> = batches.iter().map(|x| x.num_rows()).collect();
3909        assert_eq!(&batch_counts, &[2, 2, 2, 1]);
3910
3911        let actual = pretty_format_batches(&batches).unwrap().to_string();
3912        assert_eq!(actual, expected);
3913    }
3914
3915    #[test]
3916    fn test_arrow_writer_metadata() {
3917        let batch_schema = Schema::new(vec![Field::new("int32", DataType::Int32, false)]);
3918        let file_schema = batch_schema.clone().with_metadata(
3919            vec![("foo".to_string(), "bar".to_string())]
3920                .into_iter()
3921                .collect(),
3922        );
3923
3924        let batch = RecordBatch::try_new(
3925            Arc::new(batch_schema),
3926            vec![Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _],
3927        )
3928        .unwrap();
3929
3930        let mut buf = Vec::with_capacity(1024);
3931        let mut writer = ArrowWriter::try_new(&mut buf, Arc::new(file_schema), None).unwrap();
3932        writer.write(&batch).unwrap();
3933        writer.close().unwrap();
3934    }
3935
3936    #[test]
3937    fn test_arrow_writer_nullable() {
3938        let batch_schema = Schema::new(vec![Field::new("int32", DataType::Int32, false)]);
3939        let file_schema = Schema::new(vec![Field::new("int32", DataType::Int32, true)]);
3940        let file_schema = Arc::new(file_schema);
3941
3942        let batch = RecordBatch::try_new(
3943            Arc::new(batch_schema),
3944            vec![Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _],
3945        )
3946        .unwrap();
3947
3948        let mut buf = Vec::with_capacity(1024);
3949        let mut writer = ArrowWriter::try_new(&mut buf, file_schema.clone(), None).unwrap();
3950        writer.write(&batch).unwrap();
3951        writer.close().unwrap();
3952
3953        let mut read = ParquetRecordBatchReader::try_new(Bytes::from(buf), 1024).unwrap();
3954        let back = read.next().unwrap().unwrap();
3955        assert_eq!(back.schema(), file_schema);
3956        assert_ne!(back.schema(), batch.schema());
3957        assert_eq!(back.column(0).as_ref(), batch.column(0).as_ref());
3958    }
3959
3960    #[test]
3961    fn in_progress_accounting() {
3962        // define schema
3963        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
3964
3965        // create some data
3966        let a = Int32Array::from(vec![1, 2, 3, 4, 5]);
3967
3968        // build a record batch
3969        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
3970
3971        let mut writer = ArrowWriter::try_new(vec![], batch.schema(), None).unwrap();
3972
3973        // starts empty
3974        assert_eq!(writer.in_progress_size(), 0);
3975        assert_eq!(writer.in_progress_rows(), 0);
3976        assert_eq!(writer.memory_size(), 0);
3977        assert_eq!(writer.bytes_written(), 4); // Initial header
3978        writer.write(&batch).unwrap();
3979
3980        // updated on write
3981        let initial_size = writer.in_progress_size();
3982        assert!(initial_size > 0);
3983        assert_eq!(writer.in_progress_rows(), 5);
3984        let initial_memory = writer.memory_size();
3985        assert!(initial_memory > 0);
3986        // memory estimate is larger than estimated encoded size
3987        assert!(
3988            initial_size <= initial_memory,
3989            "{initial_size} <= {initial_memory}"
3990        );
3991
3992        // updated on second write
3993        writer.write(&batch).unwrap();
3994        assert!(writer.in_progress_size() > initial_size);
3995        assert_eq!(writer.in_progress_rows(), 10);
3996        assert!(writer.memory_size() > initial_memory);
3997        assert!(
3998            writer.in_progress_size() <= writer.memory_size(),
3999            "in_progress_size {} <= memory_size {}",
4000            writer.in_progress_size(),
4001            writer.memory_size()
4002        );
4003
4004        // in progress tracking is cleared, but the overall data written is updated
4005        let pre_flush_bytes_written = writer.bytes_written();
4006        writer.flush().unwrap();
4007        assert_eq!(writer.in_progress_size(), 0);
4008        assert_eq!(writer.memory_size(), 0);
4009        assert!(writer.bytes_written() > pre_flush_bytes_written);
4010
4011        writer.close().unwrap();
4012    }
4013
4014    #[test]
4015    fn test_writer_all_null() {
4016        let a = Int32Array::from(vec![1, 2, 3, 4, 5]);
4017        let b = Int32Array::new(vec![0; 5].into(), Some(NullBuffer::new_null(5)));
4018        let batch = RecordBatch::try_from_iter(vec![
4019            ("a", Arc::new(a) as ArrayRef),
4020            ("b", Arc::new(b) as ArrayRef),
4021        ])
4022        .unwrap();
4023
4024        let mut buf = Vec::with_capacity(1024);
4025        let mut writer = ArrowWriter::try_new(&mut buf, batch.schema(), None).unwrap();
4026        writer.write(&batch).unwrap();
4027        writer.close().unwrap();
4028
4029        let bytes = Bytes::from(buf);
4030        let options = ReadOptionsBuilder::new().with_page_index().build();
4031        let reader = SerializedFileReader::new_with_options(bytes, options).unwrap();
4032        let index = reader.metadata().offset_index().unwrap();
4033
4034        assert_eq!(index.len(), 1);
4035        assert_eq!(index[0].len(), 2); // 2 columns
4036        assert_eq!(index[0][0].page_locations().len(), 1); // 1 page
4037        assert_eq!(index[0][1].page_locations().len(), 1); // 1 page
4038    }
4039
4040    #[test]
4041    fn test_disabled_statistics_with_page() {
4042        let file_schema = Schema::new(vec![
4043            Field::new("a", DataType::Utf8, true),
4044            Field::new("b", DataType::Utf8, true),
4045        ]);
4046        let file_schema = Arc::new(file_schema);
4047
4048        let batch = RecordBatch::try_new(
4049            file_schema.clone(),
4050            vec![
4051                Arc::new(StringArray::from(vec!["a", "b", "c", "d"])) as _,
4052                Arc::new(StringArray::from(vec!["w", "x", "y", "z"])) as _,
4053            ],
4054        )
4055        .unwrap();
4056
4057        let props = WriterProperties::builder()
4058            .set_statistics_enabled(EnabledStatistics::None)
4059            .set_column_statistics_enabled("a".into(), EnabledStatistics::Page)
4060            .build();
4061
4062        let mut buf = Vec::with_capacity(1024);
4063        let mut writer = ArrowWriter::try_new(&mut buf, file_schema.clone(), Some(props)).unwrap();
4064        writer.write(&batch).unwrap();
4065
4066        let metadata = writer.close().unwrap();
4067        assert_eq!(metadata.num_row_groups(), 1);
4068        let row_group = metadata.row_group(0);
4069        assert_eq!(row_group.num_columns(), 2);
4070        // Column "a" has both offset and column index, as requested
4071        assert!(row_group.column(0).offset_index_offset().is_some());
4072        assert!(row_group.column(0).column_index_offset().is_some());
4073        // Column "b" should only have offset index
4074        assert!(row_group.column(1).offset_index_offset().is_some());
4075        assert!(row_group.column(1).column_index_offset().is_none());
4076
4077        let options = ReadOptionsBuilder::new().with_page_index().build();
4078        let reader = SerializedFileReader::new_with_options(Bytes::from(buf), options).unwrap();
4079
4080        let row_group = reader.get_row_group(0).unwrap();
4081        let a_col = row_group.metadata().column(0);
4082        let b_col = row_group.metadata().column(1);
4083
4084        // Column chunk of column "a" should have chunk level statistics
4085        if let Statistics::ByteArray(byte_array_stats) = a_col.statistics().unwrap() {
4086            let min = byte_array_stats.min_opt().unwrap();
4087            let max = byte_array_stats.max_opt().unwrap();
4088
4089            assert_eq!(min.as_bytes(), b"a");
4090            assert_eq!(max.as_bytes(), b"d");
4091        } else {
4092            panic!("expecting Statistics::ByteArray");
4093        }
4094
4095        // The column chunk for column "b" shouldn't have statistics
4096        assert!(b_col.statistics().is_none());
4097
4098        let offset_index = reader.metadata().offset_index().unwrap();
4099        assert_eq!(offset_index.len(), 1); // 1 row group
4100        assert_eq!(offset_index[0].len(), 2); // 2 columns
4101
4102        let column_index = reader.metadata().column_index().unwrap();
4103        assert_eq!(column_index.len(), 1); // 1 row group
4104        assert_eq!(column_index[0].len(), 2); // 2 columns
4105
4106        let a_idx = &column_index[0][0];
4107        assert!(
4108            matches!(a_idx, ColumnIndexMetaData::BYTE_ARRAY(_)),
4109            "{a_idx:?}"
4110        );
4111        let b_idx = &column_index[0][1];
4112        assert!(matches!(b_idx, ColumnIndexMetaData::NONE), "{b_idx:?}");
4113    }
4114
4115    #[test]
4116    fn test_disabled_statistics_with_chunk() {
4117        let file_schema = Schema::new(vec![
4118            Field::new("a", DataType::Utf8, true),
4119            Field::new("b", DataType::Utf8, true),
4120        ]);
4121        let file_schema = Arc::new(file_schema);
4122
4123        let batch = RecordBatch::try_new(
4124            file_schema.clone(),
4125            vec![
4126                Arc::new(StringArray::from(vec!["a", "b", "c", "d"])) as _,
4127                Arc::new(StringArray::from(vec!["w", "x", "y", "z"])) as _,
4128            ],
4129        )
4130        .unwrap();
4131
4132        let props = WriterProperties::builder()
4133            .set_statistics_enabled(EnabledStatistics::None)
4134            .set_column_statistics_enabled("a".into(), EnabledStatistics::Chunk)
4135            .build();
4136
4137        let mut buf = Vec::with_capacity(1024);
4138        let mut writer = ArrowWriter::try_new(&mut buf, file_schema.clone(), Some(props)).unwrap();
4139        writer.write(&batch).unwrap();
4140
4141        let metadata = writer.close().unwrap();
4142        assert_eq!(metadata.num_row_groups(), 1);
4143        let row_group = metadata.row_group(0);
4144        assert_eq!(row_group.num_columns(), 2);
4145        // Column "a" should only have offset index
4146        assert!(row_group.column(0).offset_index_offset().is_some());
4147        assert!(row_group.column(0).column_index_offset().is_none());
4148        // Column "b" should only have offset index
4149        assert!(row_group.column(1).offset_index_offset().is_some());
4150        assert!(row_group.column(1).column_index_offset().is_none());
4151
4152        let options = ReadOptionsBuilder::new().with_page_index().build();
4153        let reader = SerializedFileReader::new_with_options(Bytes::from(buf), options).unwrap();
4154
4155        let row_group = reader.get_row_group(0).unwrap();
4156        let a_col = row_group.metadata().column(0);
4157        let b_col = row_group.metadata().column(1);
4158
4159        // Column chunk of column "a" should have chunk level statistics
4160        if let Statistics::ByteArray(byte_array_stats) = a_col.statistics().unwrap() {
4161            let min = byte_array_stats.min_opt().unwrap();
4162            let max = byte_array_stats.max_opt().unwrap();
4163
4164            assert_eq!(min.as_bytes(), b"a");
4165            assert_eq!(max.as_bytes(), b"d");
4166        } else {
4167            panic!("expecting Statistics::ByteArray");
4168        }
4169
4170        // The column chunk for column "b"  shouldn't have statistics
4171        assert!(b_col.statistics().is_none());
4172
4173        let column_index = reader.metadata().column_index().unwrap();
4174        assert_eq!(column_index.len(), 1); // 1 row group
4175        assert_eq!(column_index[0].len(), 2); // 2 columns
4176
4177        let a_idx = &column_index[0][0];
4178        assert!(matches!(a_idx, ColumnIndexMetaData::NONE), "{a_idx:?}");
4179        let b_idx = &column_index[0][1];
4180        assert!(matches!(b_idx, ColumnIndexMetaData::NONE), "{b_idx:?}");
4181    }
4182
4183    #[test]
4184    fn test_arrow_writer_skip_metadata() {
4185        let batch_schema = Schema::new(vec![Field::new("int32", DataType::Int32, false)]);
4186        let file_schema = Arc::new(batch_schema.clone());
4187
4188        let batch = RecordBatch::try_new(
4189            Arc::new(batch_schema),
4190            vec![Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _],
4191        )
4192        .unwrap();
4193        let skip_options = ArrowWriterOptions::new().with_skip_arrow_metadata(true);
4194
4195        let mut buf = Vec::with_capacity(1024);
4196        let mut writer =
4197            ArrowWriter::try_new_with_options(&mut buf, file_schema.clone(), skip_options).unwrap();
4198        writer.write(&batch).unwrap();
4199        writer.close().unwrap();
4200
4201        let bytes = Bytes::from(buf);
4202        let reader_builder = ParquetRecordBatchReaderBuilder::try_new(bytes).unwrap();
4203        assert_eq!(file_schema, *reader_builder.schema());
4204        if let Some(key_value_metadata) = reader_builder
4205            .metadata()
4206            .file_metadata()
4207            .key_value_metadata()
4208        {
4209            assert!(
4210                !key_value_metadata
4211                    .iter()
4212                    .any(|kv| kv.key.as_str() == ARROW_SCHEMA_META_KEY)
4213            );
4214        }
4215    }
4216
4217    #[test]
4218    fn mismatched_schemas() {
4219        let batch_schema = Schema::new(vec![Field::new("count", DataType::Int32, false)]);
4220        let file_schema = Arc::new(Schema::new(vec![Field::new(
4221            "temperature",
4222            DataType::Float64,
4223            false,
4224        )]));
4225
4226        let batch = RecordBatch::try_new(
4227            Arc::new(batch_schema),
4228            vec![Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _],
4229        )
4230        .unwrap();
4231
4232        let mut buf = Vec::with_capacity(1024);
4233        let mut writer = ArrowWriter::try_new(&mut buf, file_schema.clone(), None).unwrap();
4234
4235        let err = writer.write(&batch).unwrap_err().to_string();
4236        assert_eq!(
4237            err,
4238            "Arrow: Incompatible type. Field 'temperature' has type Float64, array has type Int32"
4239        );
4240    }
4241
4242    #[test]
4243    // https://github.com/apache/arrow-rs/issues/6988
4244    fn test_roundtrip_empty_schema() {
4245        // create empty record batch with empty schema
4246        let empty_batch = RecordBatch::try_new_with_options(
4247            Arc::new(Schema::empty()),
4248            vec![],
4249            &RecordBatchOptions::default().with_row_count(Some(0)),
4250        )
4251        .unwrap();
4252
4253        // write to parquet
4254        let mut parquet_bytes: Vec<u8> = Vec::new();
4255        let mut writer =
4256            ArrowWriter::try_new(&mut parquet_bytes, empty_batch.schema(), None).unwrap();
4257        writer.write(&empty_batch).unwrap();
4258        writer.close().unwrap();
4259
4260        // read from parquet
4261        let bytes = Bytes::from(parquet_bytes);
4262        let reader = ParquetRecordBatchReaderBuilder::try_new(bytes).unwrap();
4263        assert_eq!(reader.schema(), &empty_batch.schema());
4264        let batches: Vec<_> = reader
4265            .build()
4266            .unwrap()
4267            .collect::<ArrowResult<Vec<_>>>()
4268            .unwrap();
4269        assert_eq!(batches.len(), 0);
4270    }
4271
4272    #[test]
4273    fn test_page_stats_not_written_by_default() {
4274        let string_field = Field::new("a", DataType::Utf8, false);
4275        let schema = Schema::new(vec![string_field]);
4276        let raw_string_values = vec!["Blart Versenwald III"];
4277        let string_values = StringArray::from(raw_string_values.clone());
4278        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(string_values)]).unwrap();
4279
4280        let props = WriterProperties::builder()
4281            .set_statistics_enabled(EnabledStatistics::Page)
4282            .set_dictionary_enabled(false)
4283            .set_encoding(Encoding::PLAIN)
4284            .set_compression(crate::basic::Compression::UNCOMPRESSED)
4285            .build();
4286
4287        let file = roundtrip_opts(&batch, props);
4288
4289        // read file and decode page headers
4290        // Note: use the thrift API as there is no Rust API to access the statistics in the page headers
4291
4292        // decode first page header
4293        let first_page = &file[4..];
4294        let mut prot = ThriftSliceInputProtocol::new(first_page);
4295        let hdr = PageHeader::read_thrift(&mut prot).unwrap();
4296        let stats = hdr.data_page_header.unwrap().statistics;
4297
4298        assert!(stats.is_none());
4299    }
4300
4301    #[test]
4302    fn test_page_stats_when_enabled() {
4303        let string_field = Field::new("a", DataType::Utf8, false);
4304        let schema = Schema::new(vec![string_field]);
4305        let raw_string_values = vec!["Blart Versenwald III", "Andrew Lamb"];
4306        let string_values = StringArray::from(raw_string_values.clone());
4307        let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(string_values)]).unwrap();
4308
4309        let props = WriterProperties::builder()
4310            .set_statistics_enabled(EnabledStatistics::Page)
4311            .set_dictionary_enabled(false)
4312            .set_encoding(Encoding::PLAIN)
4313            .set_write_page_header_statistics(true)
4314            .set_compression(crate::basic::Compression::UNCOMPRESSED)
4315            .build();
4316
4317        let file = roundtrip_opts(&batch, props);
4318
4319        // read file and decode page headers
4320        // Note: use the thrift API as there is no Rust API to access the statistics in the page headers
4321
4322        // decode first page header
4323        let first_page = &file[4..];
4324        let mut prot = ThriftSliceInputProtocol::new(first_page);
4325        let hdr = PageHeader::read_thrift(&mut prot).unwrap();
4326        let stats = hdr.data_page_header.unwrap().statistics;
4327
4328        let stats = stats.unwrap();
4329        // check that min/max were actually written to the page
4330        assert!(stats.is_max_value_exact.unwrap());
4331        assert!(stats.is_min_value_exact.unwrap());
4332        assert_eq!(stats.max_value.unwrap(), "Blart Versenwald III".as_bytes());
4333        assert_eq!(stats.min_value.unwrap(), "Andrew Lamb".as_bytes());
4334    }
4335
4336    #[test]
4337    fn test_page_stats_truncation() {
4338        let string_field = Field::new("a", DataType::Utf8, false);
4339        let binary_field = Field::new("b", DataType::Binary, false);
4340        let schema = Schema::new(vec![string_field, binary_field]);
4341
4342        let raw_string_values = vec!["Blart Versenwald III"];
4343        let raw_binary_values = [b"Blart Versenwald III".to_vec()];
4344        let raw_binary_value_refs = raw_binary_values
4345            .iter()
4346            .map(|x| x.as_slice())
4347            .collect::<Vec<_>>();
4348
4349        let string_values = StringArray::from(raw_string_values.clone());
4350        let binary_values = BinaryArray::from(raw_binary_value_refs);
4351        let batch = RecordBatch::try_new(
4352            Arc::new(schema),
4353            vec![Arc::new(string_values), Arc::new(binary_values)],
4354        )
4355        .unwrap();
4356
4357        let props = WriterProperties::builder()
4358            .set_statistics_truncate_length(Some(2))
4359            .set_dictionary_enabled(false)
4360            .set_encoding(Encoding::PLAIN)
4361            .set_write_page_header_statistics(true)
4362            .set_compression(crate::basic::Compression::UNCOMPRESSED)
4363            .build();
4364
4365        let file = roundtrip_opts(&batch, props);
4366
4367        // read file and decode page headers
4368        // Note: use the thrift API as there is no Rust API to access the statistics in the page headers
4369
4370        // decode first page header
4371        let first_page = &file[4..];
4372        let mut prot = ThriftSliceInputProtocol::new(first_page);
4373        let hdr = PageHeader::read_thrift(&mut prot).unwrap();
4374        let stats = hdr.data_page_header.unwrap().statistics;
4375        assert!(stats.is_some());
4376        let stats = stats.unwrap();
4377        // check that min/max were properly truncated
4378        assert!(!stats.is_max_value_exact.unwrap());
4379        assert!(!stats.is_min_value_exact.unwrap());
4380        assert_eq!(stats.max_value.unwrap(), "Bm".as_bytes());
4381        assert_eq!(stats.min_value.unwrap(), "Bl".as_bytes());
4382
4383        // check second page now
4384        let second_page = &prot.as_slice()[hdr.compressed_page_size as usize..];
4385        let mut prot = ThriftSliceInputProtocol::new(second_page);
4386        let hdr = PageHeader::read_thrift(&mut prot).unwrap();
4387        let stats = hdr.data_page_header.unwrap().statistics;
4388        assert!(stats.is_some());
4389        let stats = stats.unwrap();
4390        // check that min/max were properly truncated
4391        assert!(!stats.is_max_value_exact.unwrap());
4392        assert!(!stats.is_min_value_exact.unwrap());
4393        assert_eq!(stats.max_value.unwrap(), "Bm".as_bytes());
4394        assert_eq!(stats.min_value.unwrap(), "Bl".as_bytes());
4395    }
4396
4397    #[test]
4398    fn test_page_encoding_statistics_roundtrip() {
4399        let batch_schema = Schema::new(vec![Field::new(
4400            "int32",
4401            arrow_schema::DataType::Int32,
4402            false,
4403        )]);
4404
4405        let batch = RecordBatch::try_new(
4406            Arc::new(batch_schema.clone()),
4407            vec![Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _],
4408        )
4409        .unwrap();
4410
4411        let mut file: File = tempfile::tempfile().unwrap();
4412        let mut writer = ArrowWriter::try_new(&mut file, Arc::new(batch_schema), None).unwrap();
4413        writer.write(&batch).unwrap();
4414        let file_metadata = writer.close().unwrap();
4415
4416        assert_eq!(file_metadata.num_row_groups(), 1);
4417        assert_eq!(file_metadata.row_group(0).num_columns(), 1);
4418        assert!(
4419            file_metadata
4420                .row_group(0)
4421                .column(0)
4422                .page_encoding_stats()
4423                .is_some()
4424        );
4425        let chunk_page_stats = file_metadata
4426            .row_group(0)
4427            .column(0)
4428            .page_encoding_stats()
4429            .unwrap();
4430
4431        // check that the read metadata is also correct
4432        let options = ReadOptionsBuilder::new()
4433            .with_page_index()
4434            .with_encoding_stats_as_mask(false)
4435            .build();
4436        let reader = SerializedFileReader::new_with_options(file, options).unwrap();
4437
4438        let rowgroup = reader.get_row_group(0).expect("row group missing");
4439        assert_eq!(rowgroup.num_columns(), 1);
4440        let column = rowgroup.metadata().column(0);
4441        assert!(column.page_encoding_stats().is_some());
4442        let file_page_stats = column.page_encoding_stats().unwrap();
4443        assert_eq!(chunk_page_stats, file_page_stats);
4444    }
4445
4446    #[test]
4447    fn test_different_dict_page_size_limit() {
4448        let array = Arc::new(Int64Array::from_iter(0..1024 * 1024));
4449        let schema = Arc::new(Schema::new(vec![
4450            Field::new("col0", arrow_schema::DataType::Int64, false),
4451            Field::new("col1", arrow_schema::DataType::Int64, false),
4452        ]));
4453        let batch =
4454            arrow_array::RecordBatch::try_new(schema.clone(), vec![array.clone(), array]).unwrap();
4455
4456        let props = WriterProperties::builder()
4457            .set_dictionary_page_size_limit(1024 * 1024)
4458            .set_column_dictionary_page_size_limit(ColumnPath::from("col1"), 1024 * 1024 * 4)
4459            .build();
4460        let mut writer = ArrowWriter::try_new(Vec::new(), schema, Some(props)).unwrap();
4461        writer.write(&batch).unwrap();
4462        let data = Bytes::from(writer.into_inner().unwrap());
4463
4464        let mut metadata = ParquetMetaDataReader::new();
4465        metadata.try_parse(&data).unwrap();
4466        let metadata = metadata.finish().unwrap();
4467        let col0_meta = metadata.row_group(0).column(0);
4468        let col1_meta = metadata.row_group(0).column(1);
4469
4470        let get_dict_page_size = move |meta: &ColumnChunkMetaData| {
4471            let mut reader =
4472                SerializedPageReader::new(Arc::new(data.clone()), meta, 0, None).unwrap();
4473            let page = reader.get_next_page().unwrap().unwrap();
4474            match page {
4475                Page::DictionaryPage { buf, .. } => buf.len(),
4476                _ => panic!("expected DictionaryPage"),
4477            }
4478        };
4479
4480        assert_eq!(get_dict_page_size(col0_meta), 1024 * 1024);
4481        assert_eq!(get_dict_page_size(col1_meta), 1024 * 1024 * 4);
4482    }
4483}