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 17 commits into
base: main
Choose a base branch
from
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
250 changes: 240 additions & 10 deletions parquet/src/arrow/arrow_reader/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,15 @@

//! Contains reader which reads parquet data into arrow [`RecordBatch`]

use std::collections::VecDeque;
use std::sync::Arc;

use arrow_array::cast::AsArray;
use arrow_array::Array;
use arrow_array::{RecordBatch, RecordBatchReader};
use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef};
use arrow_select::filter::prep_null_mask_filter;
pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter};
pub use selection::{RowSelection, RowSelector};
use std::collections::VecDeque;
use std::sync::Arc;

pub use crate::arrow::array_reader::RowGroups;
use crate::arrow::array_reader::{build_array_reader, ArrayReader};
Expand All @@ -42,6 +41,8 @@ mod filter;
mod selection;
pub mod statistics;

use crate::encryption::ciphers::{CryptoContext, FileDecryptionProperties};

/// Builder for constructing parquet readers into arrow.
///
/// Most users should use one of the following specializations:
Expand Down Expand Up @@ -317,7 +318,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 +380,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 +534,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 +569,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 +578,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 @@ -677,7 +694,22 @@ impl<T: ChunkReader + 'static> Iterator for ReaderPageIterator<T> {
let total_rows = rg.num_rows() as usize;
let reader = self.reader.clone();

let ret = SerializedPageReader::new(reader, meta, total_rows, page_locations);
let crypto_context = if self.metadata.file_decryptor().is_some() {
let file_decryptor = Arc::new(self.metadata.file_decryptor().clone().unwrap());

let crypto_context = CryptoContext::new(
rg_idx,
self.column_idx,
file_decryptor.clone(),
file_decryptor,
);
Some(Arc::new(crypto_context))
} else {
None
};

let ret =
SerializedPageReader::new(reader, meta, total_rows, page_locations, crypto_context);
Some(ret.map(|x| Box::new(x) as _))
}
}
Expand Down Expand Up @@ -788,6 +820,24 @@ impl ParquetRecordBatchReader {
.build()
}

/// Create a new [`ParquetRecordBatchReader`] from the provided chunk reader and [`FileDecryptionProperties`]
///
/// Note: this is needed when the parquet file is encrypted
// todo: add options or put file_decryption_properties into options
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 +994,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 +1006,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 +1715,184 @@ mod tests {
assert!(col.value(2).is_nan());
}

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

let column_1_key = "1234567890123450".as_bytes();
let column_2_key = "1234567890123451".as_bytes();

let decryption_properties = Some(
ciphers::FileDecryptionProperties::builder()
.with_column_key("kc1".as_bytes().to_vec(), column_1_key.to_vec())
.with_column_key("kc2".as_bytes().to_vec(), column_2_key.to_vec())
.build(),
);

let metadata =
ArrowReaderMetadata::load(&file, Default::default(), decryption_properties.as_ref())
.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(), 3816);
});

let record_reader = ParquetRecordBatchReader::try_new_with_decryption(
file,
128,
decryption_properties.as_ref(),
)
.unwrap();

let mut row_count = 0;
for batch in record_reader {
let batch = batch.unwrap();
row_count += batch.num_rows();
}

assert_eq!(row_count, file_metadata.num_rows() as usize);
}

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

let footer_key = "0123456789012345".as_bytes(); // 128bit/16
let column_1_key = "1234567890123450".as_bytes();
let column_2_key = "1234567890123451".as_bytes();

let decryption_properties = Some(
ciphers::FileDecryptionProperties::builder()
.with_footer_key(footer_key.to_vec())
.with_column_key("kc1".as_bytes().to_vec(), column_1_key.to_vec())
.with_column_key("kc2".as_bytes().to_vec(), column_2_key.to_vec())
.build(),
);

let metadata =
ArrowReaderMetadata::load(&file, Default::default(), decryption_properties.as_ref())
.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 19.0.0-SNAPSHOT"
// );
}

#[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.as_ref())
.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 19.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);
});

let decryption_properties = Some(
ciphers::FileDecryptionProperties::builder()
.with_footer_key(key_code.to_vec())
.build(),
);
let record_reader = ParquetRecordBatchReader::try_new_with_decryption(
file,
128,
decryption_properties.as_ref(),
)
.unwrap();

let mut row_count = 0;
for batch in record_reader {
let batch = batch.unwrap();
row_count += batch.num_rows();

let bool_col = batch.column(0).as_boolean();
let time_col = batch
.column(1)
.as_primitive::<types::Time32MillisecondType>();
let list_col = batch.column(2).as_list::<i32>();
let timestamp_col = batch
.column(3)
.as_primitive::<types::TimestampNanosecondType>();
let f32_col = batch.column(4).as_primitive::<types::Float32Type>();
let f64_col = batch.column(5).as_primitive::<types::Float64Type>();
let binary_col = batch.column(6).as_binary::<i32>();
let fixed_size_binary_col = batch.column(7).as_fixed_size_binary();

for (i, x) in bool_col.iter().enumerate() {
assert_eq!(x.unwrap(), i % 2 == 0);
}
for (i, x) in time_col.iter().enumerate() {
assert_eq!(x.unwrap(), i as i32);
}
for (i, list_item) in list_col.iter().enumerate() {
let list_item = list_item.unwrap();
let list_item = list_item.as_primitive::<types::Int64Type>();
assert_eq!(list_item.len(), 2);
assert_eq!(list_item.value(0), ((i * 2) * 1000000000000) as i64);
assert_eq!(list_item.value(1), ((i * 2 + 1) * 1000000000000) as i64);
}
for x in timestamp_col.iter() {
assert!(x.is_some());
}
for (i, x) in f32_col.iter().enumerate() {
assert_eq!(x.unwrap(), i as f32 * 1.1f32);
}
for (i, x) in f64_col.iter().enumerate() {
assert_eq!(x.unwrap(), i as f64 * 1.1111111f64);
}
for (i, x) in binary_col.iter().enumerate() {
assert_eq!(x.is_some(), i % 2 == 0);
if let Some(x) = x {
assert_eq!(&x[0..7], b"parquet");
}
}
for (i, x) in fixed_size_binary_col.iter().enumerate() {
assert_eq!(x.unwrap(), &[i as u8; 10]);
}
}

assert_eq!(row_count, file_metadata.num_rows() as usize);
}

#[test]
fn test_read_float32_float64_byte_stream_split() {
let path = format!(
Expand Down
4 changes: 2 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,13 @@ 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)
(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)?,
ParquetMetaDataReader::decode_metadata(slice, None)?,
Some((footer_start, suffix.slice(..metadata_start))),
)
};
Expand Down
6 changes: 5 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: use file_decryption_properties
Ok(Arc::new(ParquetMetaDataReader::decode_metadata(
&buf, None,
)?))
}
.boxed()
}
Expand Down Expand Up @@ -839,6 +842,7 @@ impl RowGroups for InMemoryRowGroup<'_> {
self.metadata.column(i),
self.row_count,
page_locations,
None,
)?);

Ok(Box::new(ColumnChunkIterator {
Expand Down
Loading
Loading