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// continue to export deprecated methods until they are removed
206#[allow(deprecated)]
207pub use self::schema::arrow_to_parquet_schema;
208
209pub use self::schema::{
210 add_encoded_arrow_schema_to_metadata, encode_arrow_schema, parquet_to_arrow_field_levels,
211 parquet_to_arrow_schema, parquet_to_arrow_schema_by_columns, ArrowSchemaConverter, FieldLevels,
212};
213
214/// Schema metadata key used to store serialized Arrow schema
215///
216/// The Arrow schema is encoded using the Arrow IPC format, and then base64
217/// encoded. This is the same format used by arrow-cpp systems, such as pyarrow.
218pub const ARROW_SCHEMA_META_KEY: &str = "ARROW:schema";
219
220/// The value of this metadata key, if present on [`Field::metadata`], will be used
221/// to populate [`BasicTypeInfo::id`]
222///
223/// [`Field::metadata`]: arrow_schema::Field::metadata
224/// [`BasicTypeInfo::id`]: crate::schema::types::BasicTypeInfo::id
225pub const PARQUET_FIELD_ID_META_KEY: &str = "PARQUET:field_id";
226
227/// A [`ProjectionMask`] identifies a set of columns within a potentially nested schema to project
228///
229/// In particular, a [`ProjectionMask`] can be constructed from a list of leaf column indices
230/// or root column indices where:
231///
232/// * Root columns are the direct children of the root schema, enumerated in order
233/// * Leaf columns are the child-less leaves of the schema as enumerated by a depth-first search
234///
235/// For example, the schema
236///
237/// ```ignore
238/// message schema {
239/// REQUIRED boolean leaf_1;
240/// REQUIRED GROUP group {
241/// OPTIONAL int32 leaf_2;
242/// OPTIONAL int64 leaf_3;
243/// }
244/// }
245/// ```
246///
247/// Has roots `["leaf_1", "group"]` and leaves `["leaf_1", "leaf_2", "leaf_3"]`
248///
249/// For non-nested schemas, i.e. those containing only primitive columns, the root
250/// and leaves are the same
251///
252#[derive(Debug, Clone, PartialEq, Eq)]
253pub struct ProjectionMask {
254 /// If `Some`, a leaf column should be included if the value at
255 /// the corresponding index is true
256 ///
257 /// If `None`, all columns should be included
258 ///
259 /// # Examples
260 ///
261 /// Given the original parquet schema with leaf columns is `[a, b, c, d]`
262 ///
263 /// A mask of `[true, false, true, false]` will result in a schema 2
264 /// elements long:
265 /// * `fields[0]`: `a`
266 /// * `fields[1]`: `c`
267 ///
268 /// A mask of `None` will result in a schema 4 elements long:
269 /// * `fields[0]`: `a`
270 /// * `fields[1]`: `b`
271 /// * `fields[2]`: `c`
272 /// * `fields[3]`: `d`
273 mask: Option<Vec<bool>>,
274}
275
276impl ProjectionMask {
277 /// Create a [`ProjectionMask`] which selects all columns
278 pub fn all() -> Self {
279 Self { mask: None }
280 }
281
282 /// Create a [`ProjectionMask`] which selects only the specified leaf columns
283 ///
284 /// Note: repeated or out of order indices will not impact the final mask
285 ///
286 /// i.e. `[0, 1, 2]` will construct the same mask as `[1, 0, 0, 2]`
287 pub fn leaves(schema: &SchemaDescriptor, indices: impl IntoIterator<Item = usize>) -> Self {
288 let mut mask = vec![false; schema.num_columns()];
289 for leaf_idx in indices {
290 mask[leaf_idx] = true;
291 }
292 Self { mask: Some(mask) }
293 }
294
295 /// Create a [`ProjectionMask`] which selects only the specified root columns
296 ///
297 /// Note: repeated or out of order indices will not impact the final mask
298 ///
299 /// i.e. `[0, 1, 2]` will construct the same mask as `[1, 0, 0, 2]`
300 pub fn roots(schema: &SchemaDescriptor, indices: impl IntoIterator<Item = usize>) -> Self {
301 let num_root_columns = schema.root_schema().get_fields().len();
302 let mut root_mask = vec![false; num_root_columns];
303 for root_idx in indices {
304 root_mask[root_idx] = true;
305 }
306
307 let mask = (0..schema.num_columns())
308 .map(|leaf_idx| {
309 let root_idx = schema.get_column_root_idx(leaf_idx);
310 root_mask[root_idx]
311 })
312 .collect();
313
314 Self { mask: Some(mask) }
315 }
316
317 // Given a starting point in the schema, do a DFS for that node adding leaf paths to `paths`.
318 fn find_leaves(root: &Arc<Type>, parent: Option<&String>, paths: &mut Vec<String>) {
319 let path = parent
320 .map(|p| [p, root.name()].join("."))
321 .unwrap_or(root.name().to_string());
322 if root.is_group() {
323 for child in root.get_fields() {
324 Self::find_leaves(child, Some(&path), paths);
325 }
326 } else {
327 // Reached a leaf, add to paths
328 paths.push(path);
329 }
330 }
331
332 /// Create a [`ProjectionMask`] which selects only the named columns
333 ///
334 /// All leaf columns that fall below a given name will be selected. For example, given
335 /// the schema
336 /// ```ignore
337 /// message schema {
338 /// OPTIONAL group a (MAP) {
339 /// REPEATED group key_value {
340 /// REQUIRED BYTE_ARRAY key (UTF8); // leaf index 0
341 /// OPTIONAL group value (MAP) {
342 /// REPEATED group key_value {
343 /// REQUIRED INT32 key; // leaf index 1
344 /// REQUIRED BOOLEAN value; // leaf index 2
345 /// }
346 /// }
347 /// }
348 /// }
349 /// REQUIRED INT32 b; // leaf index 3
350 /// REQUIRED DOUBLE c; // leaf index 4
351 /// }
352 /// ```
353 /// `["a.key_value.value", "c"]` would return leaf columns 1, 2, and 4. `["a"]` would return
354 /// columns 0, 1, and 2.
355 ///
356 /// Note: repeated or out of order indices will not impact the final mask.
357 ///
358 /// i.e. `["b", "c"]` will construct the same mask as `["c", "b", "c"]`.
359 pub fn columns<'a>(
360 schema: &SchemaDescriptor,
361 names: impl IntoIterator<Item = &'a str>,
362 ) -> Self {
363 // first make vector of paths for leaf columns
364 let mut paths: Vec<String> = vec![];
365 for root in schema.root_schema().get_fields() {
366 Self::find_leaves(root, None, &mut paths);
367 }
368 assert_eq!(paths.len(), schema.num_columns());
369
370 let mut mask = vec![false; schema.num_columns()];
371 for name in names {
372 for idx in 0..schema.num_columns() {
373 if paths[idx].starts_with(name) {
374 mask[idx] = true;
375 }
376 }
377 }
378
379 Self { mask: Some(mask) }
380 }
381
382 /// Returns true if the leaf column `leaf_idx` is included by the mask
383 pub fn leaf_included(&self, leaf_idx: usize) -> bool {
384 self.mask.as_ref().map(|m| m[leaf_idx]).unwrap_or(true)
385 }
386
387 /// Union two projection masks
388 ///
389 /// Example:
390 /// ```text
391 /// mask1 = [true, false, true]
392 /// mask2 = [false, true, true]
393 /// union(mask1, mask2) = [true, true, true]
394 /// ```
395 pub fn union(&mut self, other: &Self) {
396 match (self.mask.as_ref(), other.mask.as_ref()) {
397 (None, _) | (_, None) => self.mask = None,
398 (Some(a), Some(b)) => {
399 debug_assert_eq!(a.len(), b.len());
400 let mask = a.iter().zip(b.iter()).map(|(&a, &b)| a || b).collect();
401 self.mask = Some(mask);
402 }
403 }
404 }
405
406 /// Intersect two projection masks
407 ///
408 /// Example:
409 /// ```text
410 /// mask1 = [true, false, true]
411 /// mask2 = [false, true, true]
412 /// intersect(mask1, mask2) = [false, false, true]
413 /// ```
414 pub fn intersect(&mut self, other: &Self) {
415 match (self.mask.as_ref(), other.mask.as_ref()) {
416 (None, _) => self.mask = other.mask.clone(),
417 (_, None) => {}
418 (Some(a), Some(b)) => {
419 debug_assert_eq!(a.len(), b.len());
420 let mask = a.iter().zip(b.iter()).map(|(&a, &b)| a && b).collect();
421 self.mask = Some(mask);
422 }
423 }
424 }
425}
426
427/// Lookups up the parquet column by name
428///
429/// Returns the parquet column index and the corresponding arrow field
430pub fn parquet_column<'a>(
431 parquet_schema: &SchemaDescriptor,
432 arrow_schema: &'a Schema,
433 name: &str,
434) -> Option<(usize, &'a FieldRef)> {
435 let (root_idx, field) = arrow_schema.fields.find(name)?;
436 if field.data_type().is_nested() {
437 // Nested fields are not supported and require non-trivial logic
438 // to correctly walk the parquet schema accounting for the
439 // logical type rules - <https://github.com/apache/parquet-format/blob/master/LogicalTypes.md>
440 //
441 // For example a ListArray could correspond to anything from 1 to 3 levels
442 // in the parquet schema
443 return None;
444 }
445
446 // This could be made more efficient (#TBD)
447 let parquet_idx = (0..parquet_schema.columns().len())
448 .find(|x| parquet_schema.get_column_root_idx(*x) == root_idx)?;
449 Some((parquet_idx, field))
450}
451
452#[cfg(test)]
453mod test {
454 use crate::arrow::ArrowWriter;
455 use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader, ParquetMetaDataWriter};
456 use crate::file::properties::{EnabledStatistics, WriterProperties};
457 use crate::schema::parser::parse_message_type;
458 use crate::schema::types::SchemaDescriptor;
459 use arrow_array::{ArrayRef, Int32Array, RecordBatch};
460 use bytes::Bytes;
461 use std::sync::Arc;
462
463 use super::ProjectionMask;
464
465 #[test]
466 // Reproducer for https://github.com/apache/arrow-rs/issues/6464
467 fn test_metadata_read_write_partial_offset() {
468 let parquet_bytes = create_parquet_file();
469
470 // read the metadata from the file WITHOUT the page index structures
471 let original_metadata = ParquetMetaDataReader::new()
472 .parse_and_finish(&parquet_bytes)
473 .unwrap();
474
475 // this should error because the page indexes are not present, but have offsets specified
476 let metadata_bytes = metadata_to_bytes(&original_metadata);
477 let err = ParquetMetaDataReader::new()
478 .with_page_indexes(true) // there are no page indexes in the metadata
479 .parse_and_finish(&metadata_bytes)
480 .err()
481 .unwrap();
482 assert_eq!(
483 err.to_string(),
484 "EOF: Parquet file too small. Page index range 82..115 overlaps with file metadata 0..341"
485 );
486 }
487
488 #[test]
489 fn test_metadata_read_write_roundtrip() {
490 let parquet_bytes = create_parquet_file();
491
492 // read the metadata from the file
493 let original_metadata = ParquetMetaDataReader::new()
494 .parse_and_finish(&parquet_bytes)
495 .unwrap();
496
497 // read metadata back from the serialized bytes and ensure it is the same
498 let metadata_bytes = metadata_to_bytes(&original_metadata);
499 assert_ne!(
500 metadata_bytes.len(),
501 parquet_bytes.len(),
502 "metadata is subset of parquet"
503 );
504
505 let roundtrip_metadata = ParquetMetaDataReader::new()
506 .parse_and_finish(&metadata_bytes)
507 .unwrap();
508
509 assert_eq!(original_metadata, roundtrip_metadata);
510 }
511
512 #[test]
513 fn test_metadata_read_write_roundtrip_page_index() {
514 let parquet_bytes = create_parquet_file();
515
516 // read the metadata from the file including the page index structures
517 // (which are stored elsewhere in the footer)
518 let original_metadata = ParquetMetaDataReader::new()
519 .with_page_indexes(true)
520 .parse_and_finish(&parquet_bytes)
521 .unwrap();
522
523 // read metadata back from the serialized bytes and ensure it is the same
524 let metadata_bytes = metadata_to_bytes(&original_metadata);
525 let roundtrip_metadata = ParquetMetaDataReader::new()
526 .with_page_indexes(true)
527 .parse_and_finish(&metadata_bytes)
528 .unwrap();
529
530 // Need to normalize the metadata first to remove offsets in data
531 let original_metadata = normalize_locations(original_metadata);
532 let roundtrip_metadata = normalize_locations(roundtrip_metadata);
533 assert_eq!(
534 format!("{original_metadata:#?}"),
535 format!("{roundtrip_metadata:#?}")
536 );
537 assert_eq!(original_metadata, roundtrip_metadata);
538 }
539
540 /// Sets the page index offset locations in the metadata to `None`
541 ///
542 /// This is because the offsets are used to find the relative location of the index
543 /// structures, and thus differ depending on how the structures are stored.
544 fn normalize_locations(metadata: ParquetMetaData) -> ParquetMetaData {
545 let mut metadata_builder = metadata.into_builder();
546 for rg in metadata_builder.take_row_groups() {
547 let mut rg_builder = rg.into_builder();
548 for col in rg_builder.take_columns() {
549 rg_builder = rg_builder.add_column_metadata(
550 col.into_builder()
551 .set_offset_index_offset(None)
552 .set_index_page_offset(None)
553 .set_column_index_offset(None)
554 .build()
555 .unwrap(),
556 );
557 }
558 let rg = rg_builder.build().unwrap();
559 metadata_builder = metadata_builder.add_row_group(rg);
560 }
561 metadata_builder.build()
562 }
563
564 /// Write a parquet filed into an in memory buffer
565 fn create_parquet_file() -> Bytes {
566 let mut buf = vec![];
567 let data = vec![100, 200, 201, 300, 102, 33];
568 let array: ArrayRef = Arc::new(Int32Array::from(data));
569 let batch = RecordBatch::try_from_iter(vec![("id", array)]).unwrap();
570 let props = WriterProperties::builder()
571 .set_statistics_enabled(EnabledStatistics::Page)
572 .build();
573
574 let mut writer = ArrowWriter::try_new(&mut buf, batch.schema(), Some(props)).unwrap();
575 writer.write(&batch).unwrap();
576 writer.finish().unwrap();
577 drop(writer);
578
579 Bytes::from(buf)
580 }
581
582 /// Serializes `ParquetMetaData` into a memory buffer, using `ParquetMetadataWriter
583 fn metadata_to_bytes(metadata: &ParquetMetaData) -> Bytes {
584 let mut buf = vec![];
585 ParquetMetaDataWriter::new(&mut buf, metadata)
586 .finish()
587 .unwrap();
588 Bytes::from(buf)
589 }
590
591 #[test]
592 fn test_mask_from_column_names() {
593 let message_type = "
594 message test_schema {
595 OPTIONAL group a (MAP) {
596 REPEATED group key_value {
597 REQUIRED BYTE_ARRAY key (UTF8);
598 OPTIONAL group value (MAP) {
599 REPEATED group key_value {
600 REQUIRED INT32 key;
601 REQUIRED BOOLEAN value;
602 }
603 }
604 }
605 }
606 REQUIRED INT32 b;
607 REQUIRED DOUBLE c;
608 }
609 ";
610 let parquet_group_type = parse_message_type(message_type).unwrap();
611 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
612
613 let mask = ProjectionMask::columns(&schema, ["foo", "bar"]);
614 assert_eq!(mask.mask.unwrap(), vec![false; 5]);
615
616 let mask = ProjectionMask::columns(&schema, []);
617 assert_eq!(mask.mask.unwrap(), vec![false; 5]);
618
619 let mask = ProjectionMask::columns(&schema, ["a", "c"]);
620 assert_eq!(mask.mask.unwrap(), [true, true, true, false, true]);
621
622 let mask = ProjectionMask::columns(&schema, ["a.key_value.key", "c"]);
623 assert_eq!(mask.mask.unwrap(), [true, false, false, false, true]);
624
625 let mask = ProjectionMask::columns(&schema, ["a.key_value.value", "b"]);
626 assert_eq!(mask.mask.unwrap(), [false, true, true, true, false]);
627
628 let message_type = "
629 message test_schema {
630 OPTIONAL group a (LIST) {
631 REPEATED group list {
632 OPTIONAL group element (LIST) {
633 REPEATED group list {
634 OPTIONAL group element (LIST) {
635 REPEATED group list {
636 OPTIONAL BYTE_ARRAY element (UTF8);
637 }
638 }
639 }
640 }
641 }
642 }
643 REQUIRED INT32 b;
644 }
645 ";
646 let parquet_group_type = parse_message_type(message_type).unwrap();
647 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
648
649 let mask = ProjectionMask::columns(&schema, ["a", "b"]);
650 assert_eq!(mask.mask.unwrap(), [true, true]);
651
652 let mask = ProjectionMask::columns(&schema, ["a.list.element", "b"]);
653 assert_eq!(mask.mask.unwrap(), [true, true]);
654
655 let mask =
656 ProjectionMask::columns(&schema, ["a.list.element.list.element.list.element", "b"]);
657 assert_eq!(mask.mask.unwrap(), [true, true]);
658
659 let mask = ProjectionMask::columns(&schema, ["b"]);
660 assert_eq!(mask.mask.unwrap(), [false, true]);
661
662 let message_type = "
663 message test_schema {
664 OPTIONAL INT32 a;
665 OPTIONAL INT32 b;
666 OPTIONAL INT32 c;
667 OPTIONAL INT32 d;
668 OPTIONAL INT32 e;
669 }
670 ";
671 let parquet_group_type = parse_message_type(message_type).unwrap();
672 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
673
674 let mask = ProjectionMask::columns(&schema, ["a", "b"]);
675 assert_eq!(mask.mask.unwrap(), [true, true, false, false, false]);
676
677 let mask = ProjectionMask::columns(&schema, ["d", "b", "d"]);
678 assert_eq!(mask.mask.unwrap(), [false, true, false, true, false]);
679
680 let message_type = "
681 message test_schema {
682 OPTIONAL INT32 a;
683 OPTIONAL INT32 b;
684 OPTIONAL INT32 a;
685 OPTIONAL INT32 d;
686 OPTIONAL INT32 e;
687 }
688 ";
689 let parquet_group_type = parse_message_type(message_type).unwrap();
690 let schema = SchemaDescriptor::new(Arc::new(parquet_group_type));
691
692 let mask = ProjectionMask::columns(&schema, ["a", "e"]);
693 assert_eq!(mask.mask.unwrap(), [true, false, true, false, true]);
694 }
695
696 #[test]
697 fn test_projection_mask_union() {
698 let mut mask1 = ProjectionMask {
699 mask: Some(vec![true, false, true]),
700 };
701 let mask2 = ProjectionMask {
702 mask: Some(vec![false, true, true]),
703 };
704 mask1.union(&mask2);
705 assert_eq!(mask1.mask, Some(vec![true, true, true]));
706
707 let mut mask1 = ProjectionMask { mask: None };
708 let mask2 = ProjectionMask {
709 mask: Some(vec![false, true, true]),
710 };
711 mask1.union(&mask2);
712 assert_eq!(mask1.mask, None);
713
714 let mut mask1 = ProjectionMask {
715 mask: Some(vec![true, false, true]),
716 };
717 let mask2 = ProjectionMask { mask: None };
718 mask1.union(&mask2);
719 assert_eq!(mask1.mask, None);
720
721 let mut mask1 = ProjectionMask { mask: None };
722 let mask2 = ProjectionMask { mask: None };
723 mask1.union(&mask2);
724 assert_eq!(mask1.mask, None);
725 }
726
727 #[test]
728 fn test_projection_mask_intersect() {
729 let mut mask1 = ProjectionMask {
730 mask: Some(vec![true, false, true]),
731 };
732 let mask2 = ProjectionMask {
733 mask: Some(vec![false, true, true]),
734 };
735 mask1.intersect(&mask2);
736 assert_eq!(mask1.mask, Some(vec![false, false, true]));
737
738 let mut mask1 = ProjectionMask { mask: None };
739 let mask2 = ProjectionMask {
740 mask: Some(vec![false, true, true]),
741 };
742 mask1.intersect(&mask2);
743 assert_eq!(mask1.mask, Some(vec![false, true, true]));
744
745 let mut mask1 = ProjectionMask {
746 mask: Some(vec![true, false, true]),
747 };
748 let mask2 = ProjectionMask { mask: None };
749 mask1.intersect(&mask2);
750 assert_eq!(mask1.mask, Some(vec![true, false, true]));
751
752 let mut mask1 = ProjectionMask { mask: None };
753 let mask2 = ProjectionMask { mask: None };
754 mask1.intersect(&mask2);
755 assert_eq!(mask1.mask, None);
756 }
757}