parquet/arrow/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//! API for reading/writing Arrow [`RecordBatch`]es and [`Array`]s to/from
19//! Parquet Files.
20//!
21//! See the [crate-level documentation](crate) for more details on other APIs
22//!
23//! # Schema Conversion
24//!
25//! These APIs ensure that data in Arrow [`RecordBatch`]es written to Parquet are
26//! read back as [`RecordBatch`]es with the exact same types and values.
27//!
28//! Parquet and Arrow have different type systems, and there is not
29//! always a one to one mapping between the systems. For example, data
30//! stored as a Parquet [`BYTE_ARRAY`] can be read as either an Arrow
31//! [`BinaryViewArray`] or [`BinaryArray`].
32//!
33//! To recover the original Arrow types, the writers in this module add a "hint" to
34//! the metadata in the [`ARROW_SCHEMA_META_KEY`] key which records the original Arrow
35//! schema. The metadata hint follows the same convention as arrow-cpp based
36//! implementations such as `pyarrow`. The reader looks for the schema hint in the
37//! metadata to determine Arrow types, and if it is not present, infers the Arrow schema
38//! from the Parquet schema.
39//!
40//! In situations where the embedded Arrow schema is not compatible with the Parquet
41//! schema, the Parquet schema takes precedence and no error is raised.
42//! See [#1663](https://github.com/apache/arrow-rs/issues/1663)
43//!
44//! You can also control the type conversion process in more detail using:
45//!
46//! * [`ArrowSchemaConverter`] control the conversion of Arrow types to Parquet
47//! types.
48//!
49//! * [`ArrowReaderOptions::with_schema`] to explicitly specify your own Arrow schema hint
50//! to use when reading Parquet, overriding any metadata that may be present.
51//!
52//! [`RecordBatch`]: arrow_array::RecordBatch
53//! [`Array`]: arrow_array::Array
54//! [`BYTE_ARRAY`]: crate::basic::Type::BYTE_ARRAY
55//! [`BinaryViewArray`]: arrow_array::BinaryViewArray
56//! [`BinaryArray`]: arrow_array::BinaryArray
57//! [`ArrowReaderOptions::with_schema`]: arrow_reader::ArrowReaderOptions::with_schema
58//!
59//! # Example: Writing Arrow `RecordBatch` to Parquet file
60//!
61//!```rust
62//! # use arrow_array::{Int32Array, ArrayRef};
63//! # use arrow_array::RecordBatch;
64//! # use parquet::arrow::arrow_writer::ArrowWriter;
65//! # use parquet::file::properties::WriterProperties;
66//! # use tempfile::tempfile;
67//! # use std::sync::Arc;
68//! # use parquet::basic::Compression;
69//! let ids = Int32Array::from(vec![1, 2, 3, 4]);
70//! let vals = Int32Array::from(vec![5, 6, 7, 8]);
71//! let batch = RecordBatch::try_from_iter(vec![
72//! ("id", Arc::new(ids) as ArrayRef),
73//! ("val", Arc::new(vals) as ArrayRef),
74//! ]).unwrap();
75//!
76//! let file = tempfile().unwrap();
77//!
78//! // WriterProperties can be used to set Parquet file options
79//! let props = WriterProperties::builder()
80//! .set_compression(Compression::SNAPPY)
81//! .build();
82//!
83//! let mut writer = ArrowWriter::try_new(file, batch.schema(), Some(props)).unwrap();
84//!
85//! writer.write(&batch).expect("Writing batch");
86//!
87//! // writer must be closed to write footer
88//! writer.close().unwrap();
89//! ```
90//!
91//! # Example: Reading Parquet file into Arrow `RecordBatch`
92//!
93//! ```rust
94//! # use std::fs::File;
95//! # use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder;
96//! # use std::sync::Arc;
97//! # use arrow_array::Int32Array;
98//! # use arrow::datatypes::{DataType, Field, Schema};
99//! # use arrow_array::RecordBatch;
100//! # use parquet::arrow::arrow_writer::ArrowWriter;
101//! #
102//! # let ids = Int32Array::from(vec![1, 2, 3, 4]);
103//! # let schema = Arc::new(Schema::new(vec![
104//! # Field::new("id", DataType::Int32, false),
105//! # ]));
106//! #
107//! # let file = File::create("data.parquet").unwrap();
108//! #
109//! # let batch = RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(ids)]).unwrap();
110//! # let batches = vec![batch];
111//! #
112//! # let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), None).unwrap();
113//! #
114//! # for batch in batches {
115//! # writer.write(&batch).expect("Writing batch");
116//! # }
117//! # writer.close().unwrap();
118//! #
119//! let file = File::open("data.parquet").unwrap();
120//!
121//! let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap();
122//! println!("Converted arrow schema is: {}", builder.schema());
123//!
124//! let mut reader = builder.build().unwrap();
125//!
126//! let record_batch = reader.next().unwrap().unwrap();
127//!
128//! println!("Read {} records.", record_batch.num_rows());
129//! ```
130//!
131//! # Example: Reading non-uniformly encrypted parquet file into arrow record batch
132//!
133//! Note: This requires the experimental `encryption` feature to be enabled at compile time.
134//!
135#![cfg_attr(feature = "encryption", doc = "```rust")]
136#![cfg_attr(not(feature = "encryption"), doc = "```ignore")]
137//! # use arrow_array::{Int32Array, ArrayRef};
138//! # use arrow_array::{types, RecordBatch};
139//! # use parquet::arrow::arrow_reader::{
140//! # ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReaderBuilder,
141//! # };
142//! # use arrow_array::cast::AsArray;
143//! # use parquet::file::metadata::ParquetMetaData;
144//! # use tempfile::tempfile;
145//! # use std::fs::File;
146//! # use parquet::encryption::decrypt::FileDecryptionProperties;
147//! # let test_data = arrow::util::test_util::parquet_test_data();
148//! # let path = format!("{test_data}/encrypt_columns_and_footer.parquet.encrypted");
149//! #
150//! let file = File::open(path).unwrap();
151//!
152//! // Define the AES encryption keys required required for decrypting the footer metadata
153//! // and column-specific data. If only a footer key is used then it is assumed that the
154//! // file uses uniform encryption and all columns are encrypted with the footer key.
155//! // If any column keys are specified, other columns without a key provided are assumed
156//! // to be unencrypted
157//! let footer_key = "0123456789012345".as_bytes(); // Keys are 128 bits (16 bytes)
158//! let column_1_key = "1234567890123450".as_bytes();
159//! let column_2_key = "1234567890123451".as_bytes();
160//!
161//! let decryption_properties = FileDecryptionProperties::builder(footer_key.to_vec())
162//! .with_column_key("double_field", column_1_key.to_vec())
163//! .with_column_key("float_field", column_2_key.to_vec())
164//! .build()
165//! .unwrap();
166//!
167//! let options = ArrowReaderOptions::default()
168//! .with_file_decryption_properties(decryption_properties);
169//! let reader_metadata = ArrowReaderMetadata::load(&file, options.clone()).unwrap();
170//! let file_metadata = reader_metadata.metadata().file_metadata();
171//! assert_eq!(50, file_metadata.num_rows());
172//!
173//! let mut reader = ParquetRecordBatchReaderBuilder::try_new_with_options(file, options)
174//! .unwrap()
175//! .build()
176//! .unwrap();
177//!
178//! let record_batch = reader.next().unwrap().unwrap();
179//! assert_eq!(50, record_batch.num_rows());
180//! ```
181
182experimental!(mod array_reader);
183pub mod arrow_reader;
184pub mod arrow_writer;
185mod buffer;
186mod decoder;
187
188#[cfg(feature = "async")]
189pub mod async_reader;
190#[cfg(feature = "async")]
191pub mod async_writer;
192
193mod record_reader;
194experimental!(mod schema);
195
196use std::sync::Arc;
197
198pub use self::arrow_writer::ArrowWriter;
199#[cfg(feature = "async")]
200pub use self::async_reader::ParquetRecordBatchStreamBuilder;
201#[cfg(feature = "async")]
202pub use self::async_writer::AsyncArrowWriter;
203use crate::schema::types::{SchemaDescriptor, Type};
204use arrow_schema::{FieldRef, Schema};
205
206pub use self::schema::{
207 add_encoded_arrow_schema_to_metadata, encode_arrow_schema, parquet_to_arrow_field_levels,
208 parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, ArrowSchemaConverter, FieldLevels,
209};
210
211/// Schema metadata key used to store serialized Arrow schema
212///
213/// The Arrow schema is encoded using the Arrow IPC format, and then base64
214/// encoded. This is the same format used by arrow-cpp systems, such as pyarrow.
215pub const ARROW_SCHEMA_META_KEY: &str = "ARROW:schema";
216
217/// The value of this metadata key, if present on [`Field::metadata`], will be used
218/// to populate [`BasicTypeInfo::id`]
219///
220/// [`Field::metadata`]: arrow_schema::Field::metadata
221/// [`BasicTypeInfo::id`]: crate::schema::types::BasicTypeInfo::id
222pub const PARQUET_FIELD_ID_META_KEY: &str = "PARQUET:field_id";
223
224/// A [`ProjectionMask`] identifies a set of columns within a potentially nested schema to project
225///
226/// In particular, a [`ProjectionMask`] can be constructed from a list of leaf column indices
227/// or root column indices where:
228///
229/// * Root columns are the direct children of the root schema, enumerated in order
230/// * Leaf columns are the child-less leaves of the schema as enumerated by a depth-first search
231///
232/// For example, the schema
233///
234/// ```ignore
235/// message schema {
236/// REQUIRED boolean leaf_1;
237/// REQUIRED GROUP group {
238/// OPTIONAL int32 leaf_2;
239/// OPTIONAL int64 leaf_3;
240/// }
241/// }
242/// ```
243///
244/// Has roots `["leaf_1", "group"]` and leaves `["leaf_1", "leaf_2", "leaf_3"]`
245///
246/// For non-nested schemas, i.e. those containing only primitive columns, the root
247/// and leaves are the same
248///
249#[derive(Debug, Clone, PartialEq, Eq)]
250pub struct ProjectionMask {
251 /// If `Some`, a leaf column should be included if the value at
252 /// the corresponding index is true
253 ///
254 /// If `None`, all columns should be included
255 ///
256 /// # Examples
257 ///
258 /// Given the original parquet schema with leaf columns is `[a, b, c, d]`
259 ///
260 /// A mask of `[true, false, true, false]` will result in a schema 2
261 /// elements long:
262 /// * `fields[0]`: `a`
263 /// * `fields[1]`: `c`
264 ///
265 /// A mask of `None` will result in a schema 4 elements long:
266 /// * `fields[0]`: `a`
267 /// * `fields[1]`: `b`
268 /// * `fields[2]`: `c`
269 /// * `fields[3]`: `d`
270 mask: Option<Vec<bool>>,
271}
272
273impl ProjectionMask {
274 /// Create a [`ProjectionMask`] which selects all columns
275 pub fn all() -> Self {
276 Self { mask: None }
277 }
278
279 /// Create a [`ProjectionMask`] which selects no columns
280 pub fn none(len: usize) -> Self {
281 Self {
282 mask: Some(vec![false; len]),
283 }
284 }
285
286 /// Create a [`ProjectionMask`] which selects only the specified leaf columns
287 ///
288 /// Note: repeated or out of order indices will not impact the final mask
289 ///
290 /// i.e. `[0, 1, 2]` will construct the same mask as `[1, 0, 0, 2]`
291 pub fn leaves(schema: &SchemaDescriptor, indices: impl IntoIterator<Item = usize>) -> Self {
292 let mut mask = vec![false; schema.num_columns()];
293 for leaf_idx in indices {
294 mask[leaf_idx] = true;
295 }
296 Self { mask: Some(mask) }
297 }
298
299 /// Create a [`ProjectionMask`] which selects only the specified root columns
300 ///
301 /// Note: repeated or out of order indices will not impact the final mask
302 ///
303 /// i.e. `[0, 1, 2]` will construct the same mask as `[1, 0, 0, 2]`
304 pub fn roots(schema: &SchemaDescriptor, indices: impl IntoIterator<Item = usize>) -> Self {
305 let num_root_columns = schema.root_schema().get_fields().len();
306 let mut root_mask = vec![false; num_root_columns];
307 for root_idx in indices {
308 root_mask[root_idx] = true;
309 }
310
311 let mask = (0..schema.num_columns())
312 .map(|leaf_idx| {
313 let root_idx = schema.get_column_root_idx(leaf_idx);
314 root_mask[root_idx]
315 })
316 .collect();
317
318 Self { mask: Some(mask) }
319 }
320
321 // Given a starting point in the schema, do a DFS for that node adding leaf paths to `paths`.
322 fn find_leaves(root: &Arc<Type>, parent: Option<&String>, paths: &mut Vec<String>) {
323 let path = parent
324 .map(|p| [p, root.name()].join("."))
325 .unwrap_or(root.name().to_string());
326 if root.is_group() {
327 for child in root.get_fields() {
328 Self::find_leaves(child, Some(&path), paths);
329 }
330 } else {
331 // Reached a leaf, add to paths
332 paths.push(path);
333 }
334 }
335
336 /// Create a [`ProjectionMask`] which selects only the named columns
337 ///
338 /// All leaf columns that fall below a given name will be selected. For example, given
339 /// the schema
340 /// ```ignore
341 /// message schema {
342 /// OPTIONAL group a (MAP) {
343 /// REPEATED group key_value {
344 /// REQUIRED BYTE_ARRAY key (UTF8); // leaf index 0
345 /// OPTIONAL group value (MAP) {
346 /// REPEATED group key_value {
347 /// REQUIRED INT32 key; // leaf index 1
348 /// REQUIRED BOOLEAN value; // leaf index 2
349 /// }
350 /// }
351 /// }
352 /// }
353 /// REQUIRED INT32 b; // leaf index 3
354 /// REQUIRED DOUBLE c; // leaf index 4
355 /// }
356 /// ```
357 /// `["a.key_value.value", "c"]` would return leaf columns 1, 2, and 4. `["a"]` would return
358 /// columns 0, 1, and 2.
359 ///
360 /// Note: repeated or out of order indices will not impact the final mask.
361 ///
362 /// i.e. `["b", "c"]` will construct the same mask as `["c", "b", "c"]`.
363 pub fn columns<'a>(
364 schema: &SchemaDescriptor,
365 names: impl IntoIterator<Item = &'a str>,
366 ) -> Self {
367 // first make vector of paths for leaf columns
368 let mut paths: Vec<String> = vec![];
369 for root in schema.root_schema().get_fields() {
370 Self::find_leaves(root, None, &mut paths);
371 }
372 assert_eq!(paths.len(), schema.num_columns());
373
374 let mut mask = vec![false; schema.num_columns()];
375 for name in names {
376 for idx in 0..schema.num_columns() {
377 if paths[idx].starts_with(name) {
378 mask[idx] = true;
379 }
380 }
381 }
382
383 Self { mask: Some(mask) }
384 }
385
386 /// Returns true if the leaf column `leaf_idx` is included by the mask
387 pub fn leaf_included(&self, leaf_idx: usize) -> bool {
388 self.mask.as_ref().map(|m| m[leaf_idx]).unwrap_or(true)
389 }
390
391 /// Union two projection masks
392 ///
393 /// Example:
394 /// ```text
395 /// mask1 = [true, false, true]
396 /// mask2 = [false, true, true]
397 /// union(mask1, mask2) = [true, true, true]
398 /// ```
399 pub fn union(&mut self, other: &Self) {
400 match (self.mask.as_ref(), other.mask.as_ref()) {
401 (None, _) | (_, None) => self.mask = None,
402 (Some(a), Some(b)) => {
403 debug_assert_eq!(a.len(), b.len());
404 let mask = a.iter().zip(b.iter()).map(|(&a, &b)| a || b).collect();
405 self.mask = Some(mask);
406 }
407 }
408 }
409
410 /// Intersect two projection masks
411 ///
412 /// Example:
413 /// ```text
414 /// mask1 = [true, false, true]
415 /// mask2 = [false, true, true]
416 /// intersect(mask1, mask2) = [false, false, true]
417 /// ```
418 pub fn intersect(&mut self, other: &Self) {
419 match (self.mask.as_ref(), other.mask.as_ref()) {
420 (None, _) => self.mask = other.mask.clone(),
421 (_, None) => {}
422 (Some(a), Some(b)) => {
423 debug_assert_eq!(a.len(), b.len());
424 let mask = a.iter().zip(b.iter()).map(|(&a, &b)| a && b).collect();
425 self.mask = Some(mask);
426 }
427 }
428 }
429}
430
431/// Lookups up the parquet column by name
432///
433/// Returns the parquet column index and the corresponding arrow field
434pub fn parquet_column<'a>(
435 parquet_schema: &SchemaDescriptor,
436 arrow_schema: &'a Schema,
437 name: &str,
438) -> Option<(usize, &'a FieldRef)> {
439 let (root_idx, field) = arrow_schema.fields.find(name)?;
440 if field.data_type().is_nested() {
441 // Nested fields are not supported and require non-trivial logic
442 // to correctly walk the parquet schema accounting for the
443 // logical type rules - <https://github.com/apache/parquet-format/blob/master/LogicalTypes.md>
444 //
445 // For example a ListArray could correspond to anything from 1 to 3 levels
446 // in the parquet schema
447 return None;
448 }
449
450 // This could be made more efficient (#TBD)
451 let parquet_idx = (0..parquet_schema.columns().len())
452 .find(|x| parquet_schema.get_column_root_idx(*x) == root_idx)?;
453 Some((parquet_idx, field))
454}
455
456#[cfg(test)]
457mod test {
458 use crate::arrow::ArrowWriter;
459 use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader, ParquetMetaDataWriter};
460 use crate::file::properties::{EnabledStatistics, WriterProperties};
461 use crate::schema::parser::parse_message_type;
462 use crate::schema::types::SchemaDescriptor;
463 use arrow_array::{ArrayRef, Int32Array, RecordBatch};
464 use bytes::Bytes;
465 use std::sync::Arc;
466
467 use super::ProjectionMask;
468
469 #[test]
470 // Reproducer for https://github.com/apache/arrow-rs/issues/6464
471 fn test_metadata_read_write_partial_offset() {
472 let parquet_bytes = create_parquet_file();
473
474 // read the metadata from the file WITHOUT the page index structures
475 let original_metadata = ParquetMetaDataReader::new()
476 .parse_and_finish(&parquet_bytes)
477 .unwrap();
478
479 // this should error because the page indexes are not present, but have offsets specified
480 let metadata_bytes = metadata_to_bytes(&original_metadata);
481 let err = ParquetMetaDataReader::new()
482 .with_page_indexes(true) // there are no page indexes in the metadata
483 .parse_and_finish(&metadata_bytes)
484 .err()
485 .unwrap();
486 assert_eq!(
487 err.to_string(),
488 "EOF: Parquet file too small. Page index range 82..115 overlaps with file metadata 0..357"
489 );
490 }
491
492 #[test]
493 fn test_metadata_read_write_roundtrip() {
494 let parquet_bytes = create_parquet_file();
495
496 // read the metadata from the file
497 let original_metadata = ParquetMetaDataReader::new()
498 .parse_and_finish(&parquet_bytes)
499 .unwrap();
500
501 // read metadata back from the serialized bytes and ensure it is the same
502 let metadata_bytes = metadata_to_bytes(&original_metadata);
503 assert_ne!(
504 metadata_bytes.len(),
505 parquet_bytes.len(),
506 "metadata is subset of parquet"
507 );
508
509 let roundtrip_metadata = ParquetMetaDataReader::new()
510 .parse_and_finish(&metadata_bytes)
511 .unwrap();
512
513 assert_eq!(original_metadata, roundtrip_metadata);
514 }
515
516 #[test]
517 fn test_metadata_read_write_roundtrip_page_index() {
518 let parquet_bytes = create_parquet_file();
519
520 // read the metadata from the file including the page index structures
521 // (which are stored elsewhere in the footer)
522 let original_metadata = ParquetMetaDataReader::new()
523 .with_page_indexes(true)
524 .parse_and_finish(&parquet_bytes)
525 .unwrap();
526
527 // read metadata back from the serialized bytes and ensure it is the same
528 let metadata_bytes = metadata_to_bytes(&original_metadata);
529 let roundtrip_metadata = ParquetMetaDataReader::new()
530 .with_page_indexes(true)
531 .parse_and_finish(&metadata_bytes)
532 .unwrap();
533
534 // Need to normalize the metadata first to remove offsets in data
535 let original_metadata = normalize_locations(original_metadata);
536 let roundtrip_metadata = normalize_locations(roundtrip_metadata);
537 assert_eq!(
538 format!("{original_metadata:#?}"),
539 format!("{roundtrip_metadata:#?}")
540 );
541 assert_eq!(original_metadata, roundtrip_metadata);
542 }
543
544 /// Sets the page index offset locations in the metadata to `None`
545 ///
546 /// This is because the offsets are used to find the relative location of the index
547 /// structures, and thus differ depending on how the structures are stored.
548 fn normalize_locations(metadata: ParquetMetaData) -> ParquetMetaData {
549 let mut metadata_builder = metadata.into_builder();
550 for rg in metadata_builder.take_row_groups() {
551 let mut rg_builder = rg.into_builder();
552 for col in rg_builder.take_columns() {
553 rg_builder = rg_builder.add_column_metadata(
554 col.into_builder()
555 .set_offset_index_offset(None)
556 .set_index_page_offset(None)
557 .set_column_index_offset(None)
558 .build()
559 .unwrap(),
560 );
561 }
562 let rg = rg_builder.build().unwrap();
563 metadata_builder = metadata_builder.add_row_group(rg);
564 }
565 metadata_builder.build()
566 }
567
568 /// Write a parquet filed into an in memory buffer
569 fn create_parquet_file() -> Bytes {
570 let mut buf = vec![];
571 let data = vec![100, 200, 201, 300, 102, 33];
572 let array: ArrayRef = Arc::new(Int32Array::from(data));
573 let batch = RecordBatch::try_from_iter(vec![("id", array)]).unwrap();
574 let props = WriterProperties::builder()
575 .set_statistics_enabled(EnabledStatistics::Page)
576 .set_write_page_header_statistics(true)
577 .build();
578
579 let mut writer = ArrowWriter::try_new(&mut buf, batch.schema(), Some(props)).unwrap();
580 writer.write(&batch).unwrap();
581 writer.finish().unwrap();
582 drop(writer);
583
584 Bytes::from(buf)
585 }
586
587 /// Serializes `ParquetMetaData` into a memory buffer, using `ParquetMetadataWriter
588 fn metadata_to_bytes(metadata: &ParquetMetaData) -> Bytes {
589 let mut buf = vec![];
590 ParquetMetaDataWriter::new(&mut buf, metadata)
591 .finish()
592 .unwrap();
593 Bytes::from(buf)
594 }
595
596 #[test]
597 fn test_mask_from_column_names() {
598 let message_type = "
599 message test_schema {
600 OPTIONAL group a (MAP) {
601 REPEATED group key_value {
602 REQUIRED BYTE_ARRAY key (UTF8);
603 OPTIONAL group value (MAP) {
604 REPEATED group key_value {
605 REQUIRED INT32 key;
606 REQUIRED BOOLEAN value;
607 }
608 }
609 }
610 }
611 REQUIRED INT32 b;
612 REQUIRED DOUBLE c;
613 }
614 ";
615 let parquet_group_type = parse_message_type(message_type).unwrap();
616 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
617
618 let mask = ProjectionMask::columns(&schema, ["foo", "bar"]);
619 assert_eq!(mask.mask.unwrap(), vec![false; 5]);
620
621 let mask = ProjectionMask::columns(&schema, []);
622 assert_eq!(mask.mask.unwrap(), vec![false; 5]);
623
624 let mask = ProjectionMask::columns(&schema, ["a", "c"]);
625 assert_eq!(mask.mask.unwrap(), [true, true, true, false, true]);
626
627 let mask = ProjectionMask::columns(&schema, ["a.key_value.key", "c"]);
628 assert_eq!(mask.mask.unwrap(), [true, false, false, false, true]);
629
630 let mask = ProjectionMask::columns(&schema, ["a.key_value.value", "b"]);
631 assert_eq!(mask.mask.unwrap(), [false, true, true, true, false]);
632
633 let message_type = "
634 message test_schema {
635 OPTIONAL group a (LIST) {
636 REPEATED group list {
637 OPTIONAL group element (LIST) {
638 REPEATED group list {
639 OPTIONAL group element (LIST) {
640 REPEATED group list {
641 OPTIONAL BYTE_ARRAY element (UTF8);
642 }
643 }
644 }
645 }
646 }
647 }
648 REQUIRED INT32 b;
649 }
650 ";
651 let parquet_group_type = parse_message_type(message_type).unwrap();
652 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
653
654 let mask = ProjectionMask::columns(&schema, ["a", "b"]);
655 assert_eq!(mask.mask.unwrap(), [true, true]);
656
657 let mask = ProjectionMask::columns(&schema, ["a.list.element", "b"]);
658 assert_eq!(mask.mask.unwrap(), [true, true]);
659
660 let mask =
661 ProjectionMask::columns(&schema, ["a.list.element.list.element.list.element", "b"]);
662 assert_eq!(mask.mask.unwrap(), [true, true]);
663
664 let mask = ProjectionMask::columns(&schema, ["b"]);
665 assert_eq!(mask.mask.unwrap(), [false, true]);
666
667 let message_type = "
668 message test_schema {
669 OPTIONAL INT32 a;
670 OPTIONAL INT32 b;
671 OPTIONAL INT32 c;
672 OPTIONAL INT32 d;
673 OPTIONAL INT32 e;
674 }
675 ";
676 let parquet_group_type = parse_message_type(message_type).unwrap();
677 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
678
679 let mask = ProjectionMask::columns(&schema, ["a", "b"]);
680 assert_eq!(mask.mask.unwrap(), [true, true, false, false, false]);
681
682 let mask = ProjectionMask::columns(&schema, ["d", "b", "d"]);
683 assert_eq!(mask.mask.unwrap(), [false, true, false, true, false]);
684
685 let message_type = "
686 message test_schema {
687 OPTIONAL INT32 a;
688 OPTIONAL INT32 b;
689 OPTIONAL INT32 a;
690 OPTIONAL INT32 d;
691 OPTIONAL INT32 e;
692 }
693 ";
694 let parquet_group_type = parse_message_type(message_type).unwrap();
695 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
696
697 let mask = ProjectionMask::columns(&schema, ["a", "e"]);
698 assert_eq!(mask.mask.unwrap(), [true, false, true, false, true]);
699 }
700
701 #[test]
702 fn test_projection_mask_union() {
703 let mut mask1 = ProjectionMask {
704 mask: Some(vec![true, false, true]),
705 };
706 let mask2 = ProjectionMask {
707 mask: Some(vec![false, true, true]),
708 };
709 mask1.union(&mask2);
710 assert_eq!(mask1.mask, Some(vec![true, true, true]));
711
712 let mut mask1 = ProjectionMask { mask: None };
713 let mask2 = ProjectionMask {
714 mask: Some(vec![false, true, true]),
715 };
716 mask1.union(&mask2);
717 assert_eq!(mask1.mask, None);
718
719 let mut mask1 = ProjectionMask {
720 mask: Some(vec![true, false, true]),
721 };
722 let mask2 = ProjectionMask { mask: None };
723 mask1.union(&mask2);
724 assert_eq!(mask1.mask, None);
725
726 let mut mask1 = ProjectionMask { mask: None };
727 let mask2 = ProjectionMask { mask: None };
728 mask1.union(&mask2);
729 assert_eq!(mask1.mask, None);
730 }
731
732 #[test]
733 fn test_projection_mask_intersect() {
734 let mut mask1 = ProjectionMask {
735 mask: Some(vec![true, false, true]),
736 };
737 let mask2 = ProjectionMask {
738 mask: Some(vec![false, true, true]),
739 };
740 mask1.intersect(&mask2);
741 assert_eq!(mask1.mask, Some(vec![false, false, true]));
742
743 let mut mask1 = ProjectionMask { mask: None };
744 let mask2 = ProjectionMask {
745 mask: Some(vec![false, true, true]),
746 };
747 mask1.intersect(&mask2);
748 assert_eq!(mask1.mask, Some(vec![false, true, true]));
749
750 let mut mask1 = ProjectionMask {
751 mask: Some(vec![true, false, true]),
752 };
753 let mask2 = ProjectionMask { mask: None };
754 mask1.intersect(&mask2);
755 assert_eq!(mask1.mask, Some(vec![true, false, true]));
756
757 let mut mask1 = ProjectionMask { mask: None };
758 let mask2 = ProjectionMask { mask: None };
759 mask1.intersect(&mask2);
760 assert_eq!(mask1.mask, None);
761 }
762}