From fe488b3a19cc73b3ad22ab9dcecd3cca4158156a Mon Sep 17 00:00:00 2001 From: Rok Mihevc Date: Sat, 23 Nov 2024 23:57:00 +0100 Subject: [PATCH] Fix CI --- parquet/src/arrow/arrow_reader/mod.rs | 11 ++++++++++- parquet/src/arrow/async_reader/metadata.rs | 6 ++++-- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/parquet/src/arrow/arrow_reader/mod.rs b/parquet/src/arrow/arrow_reader/mod.rs index a9190062b34..0adf9ad116f 100644 --- a/parquet/src/arrow/arrow_reader/mod.rs +++ b/parquet/src/arrow/arrow_reader/mod.rs @@ -539,6 +539,7 @@ impl ParquetRecordBatchReaderBuilder { Ok(Self::new_with_metadata(reader, metadata)) } + // Create a new [`ParquetRecordBatchReaderBuilder`] with [`ArrowReaderOptions`] and [`FileDecryptionProperties`] pub fn try_new_with_decryption( reader: T, options: ArrowReaderOptions, @@ -569,6 +570,8 @@ impl ParquetRecordBatchReaderBuilder { /// # use arrow_schema::{DataType, Field, Schema}; /// # use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder}; /// # use parquet::arrow::ArrowWriter; + /// # use crate::parquet::encryption::ciphers::FileDecryptionProperties; + /// # /// # let mut file: Vec = Vec::with_capacity(1024); /// # let schema = Arc::new(Schema::new(vec![Field::new("i32", DataType::Int32, false)])); /// # let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap(); @@ -577,7 +580,13 @@ impl ParquetRecordBatchReaderBuilder { /// # writer.close().unwrap(); /// # let file = Bytes::from(file); /// # - /// let metadata = ArrowReaderMetadata::load(&file, Default::default()).unwrap(); + /// # let key_code: &[u8] = "0123456789012345".as_bytes(); + /// let decryption_properties = Some( + /// FileDecryptionProperties::builder() + /// .with_footer_key(key_code.to_vec()) + /// .build(), + /// ); + /// let metadata = ArrowReaderMetadata::load(&file, Default::default(), decryption_properties).unwrap(); /// let mut a = ParquetRecordBatchReaderBuilder::new_with_metadata(file.clone(), metadata.clone()).build().unwrap(); /// let mut b = ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata).build().unwrap(); /// diff --git a/parquet/src/arrow/async_reader/metadata.rs b/parquet/src/arrow/async_reader/metadata.rs index b19f9830a7c..0b4584c072c 100644 --- a/parquet/src/arrow/async_reader/metadata.rs +++ b/parquet/src/arrow/async_reader/metadata.rs @@ -127,13 +127,15 @@ impl MetadataLoader { let (metadata, remainder) = if length > suffix_len - FOOTER_SIZE { let metadata_start = file_size - length - FOOTER_SIZE; let meta = fetch.fetch(metadata_start..file_size - FOOTER_SIZE).await?; - (ParquetMetaDataReader::decode_metadata(&meta)?, None) + // TODO: this won't decrypt + (ParquetMetaDataReader::decode_metadata(&meta, None)?, None) } else { let metadata_start = file_size - length - FOOTER_SIZE - footer_start; let slice = &suffix[metadata_start..suffix_len - FOOTER_SIZE]; ( - ParquetMetaDataReader::decode_metadata(slice)?, + // TODO: this won't decrypt + ParquetMetaDataReader::decode_metadata(slice, None)?, Some((footer_start, suffix.slice(..metadata_start))), ) };