parquet 56.2.1

Apache Parquet implementation in Rust
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! API for reading/writing Arrow [`RecordBatch`]es and [`Array`]s to/from
//! Parquet Files.
//!
//! See the [crate-level documentation](crate) for more details on other APIs
//!
//! # Schema Conversion
//!
//! These APIs ensure that data in Arrow [`RecordBatch`]es written to Parquet are
//! read back as [`RecordBatch`]es with the exact same types and values.
//!
//! Parquet and Arrow have different type systems, and there is not
//! always a one to one mapping between the systems. For example, data
//! stored as a Parquet [`BYTE_ARRAY`] can be read as either an Arrow
//! [`BinaryViewArray`] or [`BinaryArray`].
//!
//! To recover the original Arrow types, the writers in this module add a "hint" to
//! the metadata in the [`ARROW_SCHEMA_META_KEY`] key which records the original Arrow
//! schema. The metadata hint follows the same convention as arrow-cpp based
//! implementations such as `pyarrow`. The reader looks for the schema hint in the
//! metadata to determine Arrow types, and if it is not present, infers the Arrow schema
//! from the Parquet schema.
//!
//! In situations where the embedded Arrow schema is not compatible with the Parquet
//! schema, the Parquet schema takes precedence and no error is raised.
//! See [#1663](https://github.com/apache/arrow-rs/issues/1663)
//!
//! You can also control the type conversion process in more detail using:
//!
//! * [`ArrowSchemaConverter`] control the conversion of Arrow types to Parquet
//!   types.
//!
//! * [`ArrowReaderOptions::with_schema`] to explicitly specify your own Arrow schema hint
//!   to use when reading Parquet, overriding any metadata that may be present.
//!
//! [`RecordBatch`]: arrow_array::RecordBatch
//! [`Array`]: arrow_array::Array
//! [`BYTE_ARRAY`]: crate::basic::Type::BYTE_ARRAY
//! [`BinaryViewArray`]: arrow_array::BinaryViewArray
//! [`BinaryArray`]: arrow_array::BinaryArray
//! [`ArrowReaderOptions::with_schema`]: arrow_reader::ArrowReaderOptions::with_schema
//!
//! # Example: Writing Arrow `RecordBatch` to Parquet file
//!
//!```rust
//! # use arrow_array::{Int32Array, ArrayRef};
//! # use arrow_array::RecordBatch;
//! # use parquet::arrow::arrow_writer::ArrowWriter;
//! # use parquet::file::properties::WriterProperties;
//! # use tempfile::tempfile;
//! # use std::sync::Arc;
//! # use parquet::basic::Compression;
//! let ids = Int32Array::from(vec![1, 2, 3, 4]);
//! let vals = Int32Array::from(vec![5, 6, 7, 8]);
//! let batch = RecordBatch::try_from_iter(vec![
//!   ("id", Arc::new(ids) as ArrayRef),
//!   ("val", Arc::new(vals) as ArrayRef),
//! ]).unwrap();
//!
//! let file = tempfile().unwrap();
//!
//! // WriterProperties can be used to set Parquet file options
//! let props = WriterProperties::builder()
//!     .set_compression(Compression::SNAPPY)
//!     .build();
//!
//! let mut writer = ArrowWriter::try_new(file, batch.schema(), Some(props)).unwrap();
//!
//! writer.write(&batch).expect("Writing batch");
//!
//! // writer must be closed to write footer
//! writer.close().unwrap();
//! ```
//!
//! # Example: Reading Parquet file into Arrow `RecordBatch`
//!
//! ```rust
//! # use std::fs::File;
//! # use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder;
//! # use std::sync::Arc;
//! # use arrow_array::Int32Array;
//! # use arrow::datatypes::{DataType, Field, Schema};
//! # use arrow_array::RecordBatch;
//! # use parquet::arrow::arrow_writer::ArrowWriter;
//! #
//! # let ids = Int32Array::from(vec![1, 2, 3, 4]);
//! # let schema = Arc::new(Schema::new(vec![
//! #     Field::new("id", DataType::Int32, false),
//! # ]));
//! #
//! # let file = File::create("data.parquet").unwrap();
//! #
//! # let batch = RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(ids)]).unwrap();
//! # let batches = vec![batch];
//! #
//! # let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), None).unwrap();
//! #
//! # for batch in batches {
//! #     writer.write(&batch).expect("Writing batch");
//! # }
//! # writer.close().unwrap();
//! #
//! let file = File::open("data.parquet").unwrap();
//!
//! let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap();
//! println!("Converted arrow schema is: {}", builder.schema());
//!
//! let mut reader = builder.build().unwrap();
//!
//! let record_batch = reader.next().unwrap().unwrap();
//!
//! println!("Read {} records.", record_batch.num_rows());
//! ```
//!
//! # Example: Reading non-uniformly encrypted parquet file into arrow record batch
//!
//! Note: This requires the experimental `encryption` feature to be enabled at compile time.
//!
#![cfg_attr(feature = "encryption", doc = "```rust")]
#![cfg_attr(not(feature = "encryption"), doc = "```ignore")]
//! # use arrow_array::{Int32Array, ArrayRef};
//! # use arrow_array::{types, RecordBatch};
//! # use parquet::arrow::arrow_reader::{
//! #     ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReaderBuilder,
//! # };
//! # use arrow_array::cast::AsArray;
//! # use parquet::file::metadata::ParquetMetaData;
//! # use tempfile::tempfile;
//! # use std::fs::File;
//! # use parquet::encryption::decrypt::FileDecryptionProperties;
//! # let test_data = arrow::util::test_util::parquet_test_data();
//! # let path = format!("{test_data}/encrypt_columns_and_footer.parquet.encrypted");
//! #
//! let file = File::open(path).unwrap();
//!
//! // Define the AES encryption keys required required for decrypting the footer metadata
//! // and column-specific data. If only a footer key is used then it is assumed that the
//! // file uses uniform encryption and all columns are encrypted with the footer key.
//! // If any column keys are specified, other columns without a key provided are assumed
//! // to be unencrypted
//! let footer_key = "0123456789012345".as_bytes(); // Keys are 128 bits (16 bytes)
//! let column_1_key = "1234567890123450".as_bytes();
//! let column_2_key = "1234567890123451".as_bytes();
//!
//! let decryption_properties = FileDecryptionProperties::builder(footer_key.to_vec())
//!     .with_column_key("double_field", column_1_key.to_vec())
//!     .with_column_key("float_field", column_2_key.to_vec())
//!     .build()
//!     .unwrap();
//!
//! let options = ArrowReaderOptions::default()
//!  .with_file_decryption_properties(decryption_properties);
//! let reader_metadata = ArrowReaderMetadata::load(&file, options.clone()).unwrap();
//! let file_metadata = reader_metadata.metadata().file_metadata();
//! assert_eq!(50, file_metadata.num_rows());
//!
//! let mut reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options)
//!   .unwrap()
//!   .build()
//!   .unwrap();
//!
//! let record_batch = reader.next().unwrap().unwrap();
//! assert_eq!(50, record_batch.num_rows());
//! ```

experimental!(mod array_reader);
pub mod arrow_reader;
pub mod arrow_writer;
mod buffer;
mod decoder;

#[cfg(feature = "async")]
pub mod async_reader;
#[cfg(feature = "async")]
pub mod async_writer;

mod record_reader;
experimental!(mod schema);

use std::sync::Arc;

pub use self::arrow_writer::ArrowWriter;
#[cfg(feature = "async")]
pub use self::async_reader::ParquetRecordBatchStreamBuilder;
#[cfg(feature = "async")]
pub use self::async_writer::AsyncArrowWriter;
use crate::schema::types::{SchemaDescriptor, Type};
use arrow_schema::{FieldRef, Schema};

pub use self::schema::{
    add_encoded_arrow_schema_to_metadata, encode_arrow_schema, parquet_to_arrow_field_levels,
    parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, ArrowSchemaConverter, FieldLevels,
};

/// Schema metadata key used to store serialized Arrow schema
///
/// The Arrow schema is encoded using the Arrow IPC format, and then base64
/// encoded. This is the same format used by arrow-cpp systems, such as pyarrow.
pub const ARROW_SCHEMA_META_KEY: &str = "ARROW:schema";

/// The value of this metadata key, if present on [`Field::metadata`], will be used
/// to populate [`BasicTypeInfo::id`]
///
/// [`Field::metadata`]: arrow_schema::Field::metadata
/// [`BasicTypeInfo::id`]: crate::schema::types::BasicTypeInfo::id
pub const PARQUET_FIELD_ID_META_KEY: &str = "PARQUET:field_id";

/// A [`ProjectionMask`] identifies a set of columns within a potentially nested schema to project
///
/// In particular, a [`ProjectionMask`] can be constructed from a list of leaf column indices
/// or root column indices where:
///
/// * Root columns are the direct children of the root schema, enumerated in order
/// * Leaf columns are the child-less leaves of the schema as enumerated by a depth-first search
///
/// For example, the schema
///
/// ```ignore
/// message schema {
///   REQUIRED boolean         leaf_1;
///   REQUIRED GROUP group {
///     OPTIONAL int32 leaf_2;
///     OPTIONAL int64 leaf_3;
///   }
/// }
/// ```
///
/// Has roots `["leaf_1", "group"]` and leaves `["leaf_1", "leaf_2", "leaf_3"]`
///
/// For non-nested schemas, i.e. those containing only primitive columns, the root
/// and leaves are the same
///
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct ProjectionMask {
    /// If `Some`, a leaf column should be included if the value at
    /// the corresponding index is true
    ///
    /// If `None`, all columns should be included
    ///
    /// # Examples
    ///
    /// Given the original parquet schema with leaf columns is `[a, b, c, d]`
    ///
    /// A mask of `[true, false, true, false]` will result in a schema 2
    /// elements long:
    /// * `fields[0]`: `a`
    /// * `fields[1]`: `c`    
    ///
    /// A mask of `None` will result in a schema 4 elements long:
    /// * `fields[0]`: `a`
    /// * `fields[1]`: `b`
    /// * `fields[2]`: `c`
    /// * `fields[3]`: `d`
    mask: Option<Vec<bool>>,
}

impl ProjectionMask {
    /// Create a [`ProjectionMask`] which selects all columns
    pub fn all() -> Self {
        Self { mask: None }
    }

    /// Create a [`ProjectionMask`] which selects no columns
    pub fn none(len: usize) -> Self {
        Self {
            mask: Some(vec![false; len]),
        }
    }

    /// Create a [`ProjectionMask`] which selects only the specified leaf columns
    ///
    /// Note: repeated or out of order indices will not impact the final mask
    ///
    /// i.e. `[0, 1, 2]` will construct the same mask as `[1, 0, 0, 2]`
    pub fn leaves(schema: &SchemaDescriptor, indices: impl IntoIterator<Item = usize>) -> Self {
        let mut mask = vec![false; schema.num_columns()];
        for leaf_idx in indices {
            mask[leaf_idx] = true;
        }
        Self { mask: Some(mask) }
    }

    /// Create a [`ProjectionMask`] which selects only the specified root columns
    ///
    /// Note: repeated or out of order indices will not impact the final mask
    ///
    /// i.e. `[0, 1, 2]` will construct the same mask as `[1, 0, 0, 2]`
    pub fn roots(schema: &SchemaDescriptor, indices: impl IntoIterator<Item = usize>) -> Self {
        let num_root_columns = schema.root_schema().get_fields().len();
        let mut root_mask = vec![false; num_root_columns];
        for root_idx in indices {
            root_mask[root_idx] = true;
        }

        let mask = (0..schema.num_columns())
            .map(|leaf_idx| {
                let root_idx = schema.get_column_root_idx(leaf_idx);
                root_mask[root_idx]
            })
            .collect();

        Self { mask: Some(mask) }
    }

    // Given a starting point in the schema, do a DFS for that node adding leaf paths to `paths`.
    fn find_leaves(root: &Arc<Type>, parent: Option<&String>, paths: &mut Vec<String>) {
        let path = parent
            .map(|p| [p, root.name()].join("."))
            .unwrap_or(root.name().to_string());
        if root.is_group() {
            for child in root.get_fields() {
                Self::find_leaves(child, Some(&path), paths);
            }
        } else {
            // Reached a leaf, add to paths
            paths.push(path);
        }
    }

    /// Create a [`ProjectionMask`] which selects only the named columns
    ///
    /// All leaf columns that fall below a given name will be selected. For example, given
    /// the schema
    /// ```ignore
    /// message schema {
    ///   OPTIONAL group a (MAP) {
    ///     REPEATED group key_value {
    ///       REQUIRED BYTE_ARRAY key (UTF8);  // leaf index 0
    ///       OPTIONAL group value (MAP) {
    ///         REPEATED group key_value {
    ///           REQUIRED INT32 key;          // leaf index 1
    ///           REQUIRED BOOLEAN value;      // leaf index 2
    ///         }
    ///       }
    ///     }
    ///   }
    ///   REQUIRED INT32 b;                    // leaf index 3
    ///   REQUIRED DOUBLE c;                   // leaf index 4
    /// }
    /// ```
    /// `["a.key_value.value", "c"]` would return leaf columns 1, 2, and 4. `["a"]` would return
    /// columns 0, 1, and 2.
    ///
    /// Note: repeated or out of order indices will not impact the final mask.
    ///
    /// i.e. `["b", "c"]` will construct the same mask as `["c", "b", "c"]`.
    pub fn columns<'a>(
        schema: &SchemaDescriptor,
        names: impl IntoIterator<Item = &'a str>,
    ) -> Self {
        // first make vector of paths for leaf columns
        let mut paths: Vec<String> = vec![];
        for root in schema.root_schema().get_fields() {
            Self::find_leaves(root, None, &mut paths);
        }
        assert_eq!(paths.len(), schema.num_columns());

        let mut mask = vec![false; schema.num_columns()];
        for name in names {
            for idx in 0..schema.num_columns() {
                if paths[idx].starts_with(name) {
                    mask[idx] = true;
                }
            }
        }

        Self { mask: Some(mask) }
    }

    /// Returns true if the leaf column `leaf_idx` is included by the mask
    pub fn leaf_included(&self, leaf_idx: usize) -> bool {
        self.mask.as_ref().map(|m| m[leaf_idx]).unwrap_or(true)
    }

    /// Union two projection masks
    ///
    /// Example:
    /// ```text
    /// mask1 = [true, false, true]
    /// mask2 = [false, true, true]
    /// union(mask1, mask2) = [true, true, true]
    /// ```
    pub fn union(&mut self, other: &Self) {
        match (self.mask.as_ref(), other.mask.as_ref()) {
            (None, _) | (_, None) => self.mask = None,
            (Some(a), Some(b)) => {
                debug_assert_eq!(a.len(), b.len());
                let mask = a.iter().zip(b.iter()).map(|(&a, &b)| a || b).collect();
                self.mask = Some(mask);
            }
        }
    }

    /// Intersect two projection masks
    ///
    /// Example:
    /// ```text
    /// mask1 = [true, false, true]
    /// mask2 = [false, true, true]
    /// intersect(mask1, mask2) = [false, false, true]
    /// ```
    pub fn intersect(&mut self, other: &Self) {
        match (self.mask.as_ref(), other.mask.as_ref()) {
            (None, _) => self.mask = other.mask.clone(),
            (_, None) => {}
            (Some(a), Some(b)) => {
                debug_assert_eq!(a.len(), b.len());
                let mask = a.iter().zip(b.iter()).map(|(&a, &b)| a && b).collect();
                self.mask = Some(mask);
            }
        }
    }
}

/// Lookups up the parquet column by name
///
/// Returns the parquet column index and the corresponding arrow field
pub fn parquet_column<'a>(
    parquet_schema: &SchemaDescriptor,
    arrow_schema: &'a Schema,
    name: &str,
) -> Option<(usize, &'a FieldRef)> {
    let (root_idx, field) = arrow_schema.fields.find(name)?;
    if field.data_type().is_nested() {
        // Nested fields are not supported and require non-trivial logic
        // to correctly walk the parquet schema accounting for the
        // logical type rules - <https://github.com/apache/parquet-format/blob/master/LogicalTypes.md>
        //
        // For example a ListArray could correspond to anything from 1 to 3 levels
        // in the parquet schema
        return None;
    }

    // This could be made more efficient (#TBD)
    let parquet_idx = (0..parquet_schema.columns().len())
        .find(|x| parquet_schema.get_column_root_idx(*x) == root_idx)?;
    Some((parquet_idx, field))
}

#[cfg(test)]
mod test {
    use crate::arrow::ArrowWriter;
    use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader, ParquetMetaDataWriter};
    use crate::file::properties::{EnabledStatistics, WriterProperties};
    use crate::schema::parser::parse_message_type;
    use crate::schema::types::SchemaDescriptor;
    use arrow_array::{ArrayRef, Int32Array, RecordBatch};
    use bytes::Bytes;
    use std::sync::Arc;

    use super::ProjectionMask;

    #[test]
    #[allow(deprecated)]
    // Reproducer for https://github.com/apache/arrow-rs/issues/6464
    fn test_metadata_read_write_partial_offset() {
        let parquet_bytes = create_parquet_file();

        // read the metadata from the file WITHOUT the page index structures
        let original_metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&parquet_bytes)
            .unwrap();

        // this should error because the page indexes are not present, but have offsets specified
        let metadata_bytes = metadata_to_bytes(&original_metadata);
        let err = ParquetMetaDataReader::new()
            .with_page_indexes(true) // there are no page indexes in the metadata
            .parse_and_finish(&metadata_bytes)
            .err()
            .unwrap();
        assert_eq!(
            err.to_string(),
            "EOF: Parquet file too small. Page index range 82..115 overlaps with file metadata 0..357"
        );
    }

    #[test]
    fn test_metadata_read_write_roundtrip() {
        let parquet_bytes = create_parquet_file();

        // read the metadata from the file
        let original_metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&parquet_bytes)
            .unwrap();

        // read metadata back from the serialized bytes and ensure it is the same
        let metadata_bytes = metadata_to_bytes(&original_metadata);
        assert_ne!(
            metadata_bytes.len(),
            parquet_bytes.len(),
            "metadata is subset of parquet"
        );

        let roundtrip_metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&metadata_bytes)
            .unwrap();

        assert_eq!(original_metadata, roundtrip_metadata);
    }

    #[test]
    #[allow(deprecated)]
    fn test_metadata_read_write_roundtrip_page_index() {
        let parquet_bytes = create_parquet_file();

        // read the metadata from the file including the page index structures
        // (which are stored elsewhere in the footer)
        let original_metadata = ParquetMetaDataReader::new()
            .with_page_indexes(true)
            .parse_and_finish(&parquet_bytes)
            .unwrap();

        // read metadata back from the serialized bytes and ensure it is the same
        let metadata_bytes = metadata_to_bytes(&original_metadata);
        let roundtrip_metadata = ParquetMetaDataReader::new()
            .with_page_indexes(true)
            .parse_and_finish(&metadata_bytes)
            .unwrap();

        // Need to normalize the metadata first to remove offsets in data
        let original_metadata = normalize_locations(original_metadata);
        let roundtrip_metadata = normalize_locations(roundtrip_metadata);
        assert_eq!(
            format!("{original_metadata:#?}"),
            format!("{roundtrip_metadata:#?}")
        );
        assert_eq!(original_metadata, roundtrip_metadata);
    }

    /// Sets the page index offset locations in the metadata to `None`
    ///
    /// This is because the offsets are used to find the relative location of the index
    /// structures, and thus differ depending on how the structures are stored.
    fn normalize_locations(metadata: ParquetMetaData) -> ParquetMetaData {
        let mut metadata_builder = metadata.into_builder();
        for rg in metadata_builder.take_row_groups() {
            let mut rg_builder = rg.into_builder();
            for col in rg_builder.take_columns() {
                rg_builder = rg_builder.add_column_metadata(
                    col.into_builder()
                        .set_offset_index_offset(None)
                        .set_index_page_offset(None)
                        .set_column_index_offset(None)
                        .build()
                        .unwrap(),
                );
            }
            let rg = rg_builder.build().unwrap();
            metadata_builder = metadata_builder.add_row_group(rg);
        }
        metadata_builder.build()
    }

    /// Write a parquet filed into an in memory buffer
    fn create_parquet_file() -> Bytes {
        let mut buf = vec![];
        let data = vec![100, 200, 201, 300, 102, 33];
        let array: ArrayRef = Arc::new(Int32Array::from(data));
        let batch = RecordBatch::try_from_iter(vec![("id", array)]).unwrap();
        let props = WriterProperties::builder()
            .set_statistics_enabled(EnabledStatistics::Page)
            .set_write_page_header_statistics(true)
            .build();

        let mut writer = ArrowWriter::try_new(&mut buf, batch.schema(), Some(props)).unwrap();
        writer.write(&batch).unwrap();
        writer.finish().unwrap();
        drop(writer);

        Bytes::from(buf)
    }

    /// Serializes `ParquetMetaData` into a memory buffer, using `ParquetMetadataWriter
    fn metadata_to_bytes(metadata: &ParquetMetaData) -> Bytes {
        let mut buf = vec![];
        ParquetMetaDataWriter::new(&mut buf, metadata)
            .finish()
            .unwrap();
        Bytes::from(buf)
    }

    #[test]
    fn test_mask_from_column_names() {
        let message_type = "
            message test_schema {
                OPTIONAL group a (MAP) {
                    REPEATED group key_value {
                        REQUIRED BYTE_ARRAY key (UTF8);
                        OPTIONAL group value (MAP) {
                            REPEATED group key_value {
                                REQUIRED INT32 key;
                                REQUIRED BOOLEAN value;
                            }
                        }
                    }
                }
                REQUIRED INT32 b;
                REQUIRED DOUBLE c;
            }
            ";
        let parquet_group_type = parse_message_type(message_type).unwrap();
        let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));

        let mask = ProjectionMask::columns(&schema, ["foo", "bar"]);
        assert_eq!(mask.mask.unwrap(), vec![false; 5]);

        let mask = ProjectionMask::columns(&schema, []);
        assert_eq!(mask.mask.unwrap(), vec![false; 5]);

        let mask = ProjectionMask::columns(&schema, ["a", "c"]);
        assert_eq!(mask.mask.unwrap(), [true, true, true, false, true]);

        let mask = ProjectionMask::columns(&schema, ["a.key_value.key", "c"]);
        assert_eq!(mask.mask.unwrap(), [true, false, false, false, true]);

        let mask = ProjectionMask::columns(&schema, ["a.key_value.value", "b"]);
        assert_eq!(mask.mask.unwrap(), [false, true, true, true, false]);

        let message_type = "
            message test_schema {
                OPTIONAL group a (LIST) {
                    REPEATED group list {
                        OPTIONAL group element (LIST) {
                            REPEATED group list {
                                OPTIONAL group element (LIST) {
                                    REPEATED group list {
                                        OPTIONAL BYTE_ARRAY element (UTF8);
                                    }
                                }
                            }
                        }
                    }
                }
                REQUIRED INT32 b;
            }
            ";
        let parquet_group_type = parse_message_type(message_type).unwrap();
        let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));

        let mask = ProjectionMask::columns(&schema, ["a", "b"]);
        assert_eq!(mask.mask.unwrap(), [true, true]);

        let mask = ProjectionMask::columns(&schema, ["a.list.element", "b"]);
        assert_eq!(mask.mask.unwrap(), [true, true]);

        let mask =
            ProjectionMask::columns(&schema, ["a.list.element.list.element.list.element", "b"]);
        assert_eq!(mask.mask.unwrap(), [true, true]);

        let mask = ProjectionMask::columns(&schema, ["b"]);
        assert_eq!(mask.mask.unwrap(), [false, true]);

        let message_type = "
            message test_schema {
                OPTIONAL INT32 a;
                OPTIONAL INT32 b;
                OPTIONAL INT32 c;
                OPTIONAL INT32 d;
                OPTIONAL INT32 e;
            }
            ";
        let parquet_group_type = parse_message_type(message_type).unwrap();
        let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));

        let mask = ProjectionMask::columns(&schema, ["a", "b"]);
        assert_eq!(mask.mask.unwrap(), [true, true, false, false, false]);

        let mask = ProjectionMask::columns(&schema, ["d", "b", "d"]);
        assert_eq!(mask.mask.unwrap(), [false, true, false, true, false]);

        let message_type = "
            message test_schema {
                OPTIONAL INT32 a;
                OPTIONAL INT32 b;
                OPTIONAL INT32 a;
                OPTIONAL INT32 d;
                OPTIONAL INT32 e;
            }
            ";
        let parquet_group_type = parse_message_type(message_type).unwrap();
        let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));

        let mask = ProjectionMask::columns(&schema, ["a", "e"]);
        assert_eq!(mask.mask.unwrap(), [true, false, true, false, true]);
    }

    #[test]
    fn test_projection_mask_union() {
        let mut mask1 = ProjectionMask {
            mask: Some(vec![true, false, true]),
        };
        let mask2 = ProjectionMask {
            mask: Some(vec![false, true, true]),
        };
        mask1.union(&mask2);
        assert_eq!(mask1.mask, Some(vec![true, true, true]));

        let mut mask1 = ProjectionMask { mask: None };
        let mask2 = ProjectionMask {
            mask: Some(vec![false, true, true]),
        };
        mask1.union(&mask2);
        assert_eq!(mask1.mask, None);

        let mut mask1 = ProjectionMask {
            mask: Some(vec![true, false, true]),
        };
        let mask2 = ProjectionMask { mask: None };
        mask1.union(&mask2);
        assert_eq!(mask1.mask, None);

        let mut mask1 = ProjectionMask { mask: None };
        let mask2 = ProjectionMask { mask: None };
        mask1.union(&mask2);
        assert_eq!(mask1.mask, None);
    }

    #[test]
    fn test_projection_mask_intersect() {
        let mut mask1 = ProjectionMask {
            mask: Some(vec![true, false, true]),
        };
        let mask2 = ProjectionMask {
            mask: Some(vec![false, true, true]),
        };
        mask1.intersect(&mask2);
        assert_eq!(mask1.mask, Some(vec![false, false, true]));

        let mut mask1 = ProjectionMask { mask: None };
        let mask2 = ProjectionMask {
            mask: Some(vec![false, true, true]),
        };
        mask1.intersect(&mask2);
        assert_eq!(mask1.mask, Some(vec![false, true, true]));

        let mut mask1 = ProjectionMask {
            mask: Some(vec![true, false, true]),
        };
        let mask2 = ProjectionMask { mask: None };
        mask1.intersect(&mask2);
        assert_eq!(mask1.mask, Some(vec![true, false, true]));

        let mut mask1 = ProjectionMask { mask: None };
        let mask2 = ProjectionMask { mask: None };
        mask1.intersect(&mask2);
        assert_eq!(mask1.mask, None);
    }
}