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