Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Parquet Modular Encryption support #6637

Draft
wants to merge 22 commits into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions parquet/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ paste = { version = "1.0" }
half = { version = "2.1", default-features = false, features = ["num-traits"] }
sysinfo = { version = "0.32.0", optional = true, default-features = false, features = ["system"] }
crc32fast = { version = "1.4.2", optional = true, default-features = false }
ring = { version = "0.17", default-features = false, features = ["std"]}

[dev-dependencies]
base64 = { version = "0.22", default-features = false, features = ["std"] }
Expand Down Expand Up @@ -120,6 +121,7 @@ zstd = ["dep:zstd", "zstd-sys"]
sysinfo = ["dep:sysinfo"]
# Verify 32-bit CRC checksum when decoding parquet pages
crc = ["dep:crc32fast"]
#encryption = ["aes-gcm", "base64"]


[[example]]
Expand Down
81 changes: 75 additions & 6 deletions parquet/src/arrow/arrow_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@ mod filter;
mod selection;
pub mod statistics;

use crate::encryption::ciphers::FileDecryptionProperties;

/// Builder for constructing parquet readers into arrow.
///
/// Most users should use one of the following specializations:
Expand Down Expand Up @@ -317,7 +319,7 @@ impl ArrowReaderOptions {
///
/// // Create the reader and read the data using the supplied schema.
/// let mut reader = builder.build().unwrap();
/// let _batch = reader.next().unwrap().unwrap();
/// let _batch = reader.next().unwrap().unwrap();
/// ```
pub fn with_schema(self, schema: SchemaRef) -> Self {
Self {
Expand Down Expand Up @@ -379,9 +381,14 @@ impl ArrowReaderMetadata {
/// If `options` has [`ArrowReaderOptions::with_page_index`] true, but
/// `Self::metadata` is missing the page index, this function will attempt
/// to load the page index by making an object store request.
pub fn load<T: ChunkReader>(reader: &T, options: ArrowReaderOptions) -> Result<Self> {
pub fn load<T: ChunkReader>(
reader: &T,
options: ArrowReaderOptions,
file_decryption_properties: Option<FileDecryptionProperties>,
) -> Result<Self> {
let metadata = ParquetMetaDataReader::new()
.with_page_indexes(options.page_index)
.with_encryption_properties(file_decryption_properties)
.parse_and_finish(reader)?;
Self::try_new(Arc::new(metadata), options)
}
Expand Down Expand Up @@ -528,7 +535,17 @@ impl<T: ChunkReader + 'static> ParquetRecordBatchReaderBuilder<T> {

/// Create a new [`ParquetRecordBatchReaderBuilder`] with [`ArrowReaderOptions`]
pub fn try_new_with_options(reader: T, options: ArrowReaderOptions) -> Result<Self> {
let metadata = ArrowReaderMetadata::load(&reader, options)?;
let metadata = ArrowReaderMetadata::load(&reader, options, None)?;
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,
file_decryption_properties: Option<FileDecryptionProperties>,
) -> Result<Self> {
let metadata = ArrowReaderMetadata::load(&reader, options, file_decryption_properties)?;
Ok(Self::new_with_metadata(reader, metadata))
}

Expand All @@ -553,6 +570,7 @@ impl<T: ChunkReader + 'static> ParquetRecordBatchReaderBuilder<T> {
/// # use arrow_schema::{DataType, Field, Schema};
/// # use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder};
/// # use parquet::arrow::ArrowWriter;
/// #
/// # let mut file: Vec<u8> = 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();
Expand All @@ -561,7 +579,7 @@ impl<T: ChunkReader + 'static> ParquetRecordBatchReaderBuilder<T> {
/// # writer.close().unwrap();
/// # let file = Bytes::from(file);
/// #
/// let metadata = ArrowReaderMetadata::load(&file, Default::default()).unwrap();
/// let metadata = ArrowReaderMetadata::load(&file, Default::default(), None).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();
///
Expand Down Expand Up @@ -788,6 +806,23 @@ impl ParquetRecordBatchReader {
.build()
}

/// Create a new [`ParquetRecordBatchReader`] from the provided chunk reader and [`FileDecryptionProperties`]
///
/// Note: this is needed when the parquet file is encrypted
pub fn try_new_with_decryption<T: ChunkReader + 'static>(
reader: T,
batch_size: usize,
file_decryption_properties: Option<FileDecryptionProperties>,
) -> Result<Self> {
ParquetRecordBatchReaderBuilder::try_new_with_decryption(
reader,
Default::default(),
file_decryption_properties,
)?
.with_batch_size(batch_size)
.build()
}

/// Create a new [`ParquetRecordBatchReader`] from the provided [`RowGroups`]
///
/// Note: this is a low-level interface see [`ParquetRecordBatchReader::try_new`] for a
Expand Down Expand Up @@ -944,8 +979,9 @@ mod tests {
use arrow_select::concat::concat_batches;

use crate::arrow::arrow_reader::{
ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReader,
ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector,
ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderMetadata, ArrowReaderOptions,
ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder, RowFilter, RowSelection,
RowSelector,
};
use crate::arrow::schema::add_encoded_arrow_schema_to_metadata;
use crate::arrow::{ArrowWriter, ProjectionMask};
Expand All @@ -955,6 +991,7 @@ mod tests {
BoolType, ByteArray, ByteArrayType, DataType, FixedLenByteArray, FixedLenByteArrayType,
FloatType, Int32Type, Int64Type, Int96Type,
};
use crate::encryption::ciphers;
use crate::errors::Result;
use crate::file::properties::{EnabledStatistics, WriterProperties, WriterVersion};
use crate::file::writer::SerializedFileWriter;
Expand Down Expand Up @@ -1663,6 +1700,38 @@ mod tests {
assert!(col.value(2).is_nan());
}

#[test]
fn test_uniform_encryption() {
let testdata = arrow::util::test_util::parquet_test_data();
let path = format!("{testdata}/uniform_encryption.parquet.encrypted");
let file = File::open(path).unwrap();

let key_code: &[u8] = "0123456789012345".as_bytes();
let decryption_properties = Some(
ciphers::FileDecryptionProperties::builder()
.with_footer_key(key_code.to_vec())
.build(),
);

let metadata = ArrowReaderMetadata::load(&file, Default::default(), decryption_properties.clone()).unwrap();
let file_metadata = metadata.metadata.file_metadata();

assert_eq!(file_metadata.num_rows(), 50);
assert_eq!(file_metadata.schema_descr().num_columns(), 8);
assert_eq!(file_metadata.created_by().unwrap(), "parquet-cpp-arrow version 14.0.0-SNAPSHOT");

metadata.metadata.row_groups().iter().for_each(|rg| {
assert_eq!(rg.num_columns(), 8);
assert_eq!(rg.num_rows(), 50);
assert_eq!(rg.total_byte_size(), 4172);
});

// todo: decrypting data
// let record_reader =
// ParquetRecordBatchReader::try_new_with_decryption(file, 128, decryption_properties)
// .unwrap();
}

#[test]
fn test_read_float32_float64_byte_stream_split() {
let path = format!(
Expand Down
6 changes: 4 additions & 2 deletions parquet/src/arrow/async_reader/metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -127,13 +127,15 @@ impl<F: MetadataFetch> MetadataLoader<F> {
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))),
)
};
Expand Down
5 changes: 4 additions & 1 deletion parquet/src/arrow/async_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,10 @@ impl<T: AsyncRead + AsyncSeek + Unpin + Send> AsyncFileReader for T {
let mut buf = Vec::with_capacity(metadata_len);
self.take(metadata_len as _).read_to_end(&mut buf).await?;

Ok(Arc::new(ParquetMetaDataReader::decode_metadata(&buf)?))
// TODO: add self.file_decryption_properties
Ok(Arc::new(ParquetMetaDataReader::decode_metadata(
&buf, None,
)?))
}
.boxed()
}
Expand Down
Loading
Loading