Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-3025176564 Thank you @corwinjoy -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-3021088650 Thanks @alamb much appreciated for the review and helpful feedback! We hope to have a followup PR soon with a config to make encryption optional. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-3015285765 Thanks again @corwinjoy / @adamreeve and everyone else. This is great -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb merged PR #16351: URL: https://github.com/apache/datafusion/pull/16351 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2168131023
##
datafusion-cli/tests/sql/encrypted_parquet.sql:
##
@@ -0,0 +1,75 @@
+/*
+Test parquet encryption and decryption in DataFusion SQL.
+See datafusion/common/src/config.rs for equivalent rust code
+*/
+
+-- Keys are hex encoded, you can generate these via encode, e.g.
+select encode('0123456789012345', 'hex');
+/*
+Expected output:
++--+
+| encode(Utf8("0123456789012345"),Utf8("hex")) |
++--+
+| 30313233343536373839303132333435 |
++--+
+*/
+
+CREATE EXTERNAL TABLE encrypted_parquet_table
+(
+double_field double,
+float_field float
+)
+STORED AS PARQUET LOCATION 'pq/' OPTIONS (
+'format.crypto.file_encryption.encrypt_footer' 'true',
+'format.crypto.file_encryption.footer_key_as_hex'
'30313233343536373839303132333435', -- b"0123456789012345"
+'format.crypto.file_encryption.column_key_as_hex::double_field'
'31323334353637383930313233343530', -- b"1234567890123450"
+'format.crypto.file_encryption.column_key_as_hex::float_field'
'31323334353637383930313233343531', -- b"1234567890123451"
+ -- Same for decryption
+'format.crypto.file_decryption.footer_key_as_hex'
'30313233343536373839303132333435', -- b"0123456789012345"
+'format.crypto.file_decryption.column_key_as_hex::double_field'
'31323334353637383930313233343530', -- b"1234567890123450"
+'format.crypto.file_decryption.column_key_as_hex::float_field'
'31323334353637383930313233343531', -- b"1234567890123451"
+);
+
+CREATE TABLE temp_table (
+double_field double,
+float_field float
+);
+
+INSERT INTO temp_table VALUES(-1.0, -1.0);
+INSERT INTO temp_table VALUES(1.0, 2.0);
+INSERT INTO temp_table VALUES(3.0, 4.0);
+INSERT INTO temp_table VALUES(5.0, 6.0);
+
+INSERT INTO TABLE encrypted_parquet_table(double_field, float_field) SELECT *
FROM temp_table;
+
+SELECT * FROM encrypted_parquet_table
+WHERE double_field > 0.0 AND float_field > 0.0;
+
+/*
+Expected output:
++--+-+
+| double_field | float_field |
++--+-+
+| 1.0 | 2.0 |
+| 5.0 | 6.0 |
+| 3.0 | 4.0 |
++--+-+
+*/
+
+CREATE EXTERNAL TABLE parquet_table
+(
+double_field double,
+float_field float
+)
+STORED AS PARQUET LOCATION 'pq/';
+
+SELECT * FROM parquet_table;
+/*
+Expected output:
+Parquet error: Parquet error: Parquet file has an encrypted footer but
decryption properties were not provided
+*/
+
+
Review Comment:
Thanks! I have removed this file and added it as a test.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2168097037
##
datafusion/common/src/config.rs:
##
@@ -2017,6 +2056,305 @@ config_namespace_with_hashmap! {
}
}
+#[derive(Clone, Debug, Default, PartialEq)]
+pub struct ConfigFileEncryptionProperties {
+/// Should the parquet footer be encrypted
+pub encrypt_footer: bool,
Review Comment:
OK. Added.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351: URL: https://github.com/apache/datafusion/pull/16351#discussion_r2168085502 ## docs/source/user-guide/configs.md: ## @@ -81,6 +81,8 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | | datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.file_decryption_properties | NULL | Optional file decryption properties | Review Comment: Thanks! Docs added to `docs/source/user-guide/cli/datasources.md` which explains parquet specific options and gives an example. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2168078769
##
datafusion/proto-common/src/from_proto/mod.rs:
##
@@ -1066,6 +1066,7 @@ impl TryFrom<&protobuf::TableParquetOptions> for
TableParquetOptions {
.unwrap(),
column_specific_options,
key_value_metadata: Default::default(),
+crypto: Default::default(),
Review Comment:
Should we? I see that `key_value_metadata` is always set to empty here
despite being defined in datafusion/proto-common/src/generated/prost.rs. So,
it's not clear to me that we should throw an error here.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
adamreeve commented on code in PR #16351: URL: https://github.com/apache/datafusion/pull/16351#discussion_r2167900882 ## datafusion/datasource-parquet/src/file_format.rs: ## @@ -930,12 +959,14 @@ pub async fn fetch_parquet_metadata( store: &dyn ObjectStore, meta: &ObjectMeta, size_hint: Option, +decryption_properties: Option<&FileDecryptionProperties>, Review Comment: I guess this probably needs to change anyway to handle having an `encryption` feature. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
adamreeve commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2167883986
##
datafusion/datasource-parquet/src/file_format.rs:
##
@@ -930,12 +959,14 @@ pub async fn fetch_parquet_metadata(
store: &dyn ObjectStore,
meta: &ObjectMeta,
size_hint: Option,
+decryption_properties: Option<&FileDecryptionProperties>,
Review Comment:
This looks like a breaking change (and same for `fetch_statistics`). Should
we introduce a new method with this parameter added to avoid breaking the API?
It is documented as "subject to change" though so maybe this is ok...
##
datafusion/common/src/config.rs:
##
@@ -2017,6 +2056,305 @@ config_namespace_with_hashmap! {
}
}
+#[derive(Clone, Debug, Default, PartialEq)]
+pub struct ConfigFileEncryptionProperties {
+/// Should the parquet footer be encrypted
+pub encrypt_footer: bool,
Review Comment:
Footer encryption should probably be enabled by default, which I think means
we need to explicitly implement the Default trait.
##
datafusion/proto-common/src/from_proto/mod.rs:
##
@@ -1066,6 +1066,7 @@ impl TryFrom<&protobuf::TableParquetOptions> for
TableParquetOptions {
.unwrap(),
column_specific_options,
key_value_metadata: Default::default(),
+crypto: Default::default(),
Review Comment:
If we aren't going to support conversion to protobuf yet, we should probably
raise an error in `impl TryFrom<&TableParquetOptions> for
protobuf::TableParquetOptions` if any crypto options are set.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on code in PR #16351: URL: https://github.com/apache/datafusion/pull/16351#discussion_r2167542855 ## docs/source/user-guide/configs.md: ## @@ -81,6 +81,8 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | | datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.file_decryption_properties | NULL | Optional file decryption properties | Review Comment: I think adding an example to https://github.com/apache/datafusion/blob/main/docs/source/user-guide/cli/usage.md would make sense That would put them on https://datafusion.apache.org/user-guide/cli/usage.html -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2167546893
##
datafusion-cli/tests/sql/encrypted_parquet.sql:
##
@@ -0,0 +1,75 @@
+/*
+Test parquet encryption and decryption in DataFusion SQL.
+See datafusion/common/src/config.rs for equivalent rust code
+*/
+
+-- Keys are hex encoded, you can generate these via encode, e.g.
+select encode('0123456789012345', 'hex');
+/*
+Expected output:
++--+
+| encode(Utf8("0123456789012345"),Utf8("hex")) |
++--+
+| 30313233343536373839303132333435 |
++--+
+*/
+
+CREATE EXTERNAL TABLE encrypted_parquet_table
+(
+double_field double,
+float_field float
+)
+STORED AS PARQUET LOCATION 'pq/' OPTIONS (
+'format.crypto.file_encryption.encrypt_footer' 'true',
+'format.crypto.file_encryption.footer_key_as_hex'
'30313233343536373839303132333435', -- b"0123456789012345"
+'format.crypto.file_encryption.column_key_as_hex::double_field'
'31323334353637383930313233343530', -- b"1234567890123450"
+'format.crypto.file_encryption.column_key_as_hex::float_field'
'31323334353637383930313233343531', -- b"1234567890123451"
+ -- Same for decryption
+'format.crypto.file_decryption.footer_key_as_hex'
'30313233343536373839303132333435', -- b"0123456789012345"
+'format.crypto.file_decryption.column_key_as_hex::double_field'
'31323334353637383930313233343530', -- b"1234567890123450"
+'format.crypto.file_decryption.column_key_as_hex::float_field'
'31323334353637383930313233343531', -- b"1234567890123451"
+);
+
+CREATE TABLE temp_table (
+double_field double,
+float_field float
+);
+
+INSERT INTO temp_table VALUES(-1.0, -1.0);
+INSERT INTO temp_table VALUES(1.0, 2.0);
+INSERT INTO temp_table VALUES(3.0, 4.0);
+INSERT INTO temp_table VALUES(5.0, 6.0);
+
+INSERT INTO TABLE encrypted_parquet_table(double_field, float_field) SELECT *
FROM temp_table;
+
+SELECT * FROM encrypted_parquet_table
+WHERE double_field > 0.0 AND float_field > 0.0;
+
+/*
+Expected output:
++--+-+
+| double_field | float_field |
++--+-+
+| 1.0 | 2.0 |
+| 5.0 | 6.0 |
+| 3.0 | 4.0 |
++--+-+
+*/
+
+CREATE EXTERNAL TABLE parquet_table
+(
+double_field double,
+float_field float
+)
+STORED AS PARQUET LOCATION 'pq/';
+
+SELECT * FROM parquet_table;
+/*
+Expected output:
+Parquet error: Parquet error: Parquet file has an encrypted footer but
decryption properties were not provided
+*/
+
+
Review Comment:
I would recommend using sqllogictest to write sql reading/writing tests for
this feature.
Here are the instructions:
https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest
Ideally you should be able to add a new slt file in:
https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest/test_files
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-3003280082 > Thank you @corwinjoy and @adamreeve -- this PR was a joy to read and review. The code is clear, well commented, and well tested ❤️ 🏆 > > I think we should follow up with: > > 1. Improve the documentation to include the format required for encryption/decryption properties > 2. Consider adding a `encyrption` or similar feature flag so people who don't want support for parquet encryption can avoid bringing along the dependencies Thanks @alamb !! I have added a commit to improve the documentation and add a CLI example. (I could use some feedback on where to put the CLI example.) I will start looking into adding an `encryption` feature tomorrow. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2165756822
##
datafusion-cli/tests/sql/encrypted_parquet.sql:
##
@@ -0,0 +1,75 @@
+/*
+Test parquet encryption and decryption in DataFusion SQL.
+See datafusion/common/src/config.rs for equivalent rust code
+*/
+
+-- Keys are hex encoded, you can generate these via encode, e.g.
+select encode('0123456789012345', 'hex');
+/*
+Expected output:
++--+
+| encode(Utf8("0123456789012345"),Utf8("hex")) |
++--+
+| 30313233343536373839303132333435 |
++--+
+*/
+
+CREATE EXTERNAL TABLE encrypted_parquet_table
+(
+double_field double,
+float_field float
+)
+STORED AS PARQUET LOCATION 'pq/' OPTIONS (
+'format.crypto.file_encryption.encrypt_footer' 'true',
+'format.crypto.file_encryption.footer_key_as_hex'
'30313233343536373839303132333435', -- b"0123456789012345"
+'format.crypto.file_encryption.column_key_as_hex::double_field'
'31323334353637383930313233343530', -- b"1234567890123450"
+'format.crypto.file_encryption.column_key_as_hex::float_field'
'31323334353637383930313233343531', -- b"1234567890123451"
+ -- Same for decryption
+'format.crypto.file_decryption.footer_key_as_hex'
'30313233343536373839303132333435', -- b"0123456789012345"
+'format.crypto.file_decryption.column_key_as_hex::double_field'
'31323334353637383930313233343530', -- b"1234567890123450"
+'format.crypto.file_decryption.column_key_as_hex::float_field'
'31323334353637383930313233343531', -- b"1234567890123451"
+);
+
+CREATE TABLE temp_table (
+double_field double,
+float_field float
+);
+
+INSERT INTO temp_table VALUES(-1.0, -1.0);
+INSERT INTO temp_table VALUES(1.0, 2.0);
+INSERT INTO temp_table VALUES(3.0, 4.0);
+INSERT INTO temp_table VALUES(5.0, 6.0);
+
+INSERT INTO TABLE encrypted_parquet_table(double_field, float_field) SELECT *
FROM temp_table;
+
+SELECT * FROM encrypted_parquet_table
+WHERE double_field > 0.0 AND float_field > 0.0;
+
+/*
+Expected output:
++--+-+
+| double_field | float_field |
++--+-+
+| 1.0 | 2.0 |
+| 5.0 | 6.0 |
+| 3.0 | 4.0 |
++--+-+
+*/
+
+CREATE EXTERNAL TABLE parquet_table
+(
+double_field double,
+float_field float
+)
+STORED AS PARQUET LOCATION 'pq/';
+
+SELECT * FROM parquet_table;
+/*
+Expected output:
+Parquet error: Parquet error: Parquet file has an encrypted footer but
decryption properties were not provided
+*/
+
+
Review Comment:
I could use a little help here. I think it makes sense to add a CLI example
for how to use encryption properties. And / or there should be a test of the
CLI with these properties. However, the tests seem to only be basic SQL so this
doesn't really fit. So maybe this should just be an example that I put
somewhere else?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2165246656
##
docs/source/user-guide/configs.md:
##
@@ -81,6 +81,8 @@ Environment variables are read during `SessionConfig`
initialisation so they mus
| datafusion.execution.parquet.allow_single_file_parallelism |
true | (writing) Controls whether DataFusion will attempt
to speed up writing parquet files by serializing them in parallel. Each column
in each row group in each output file are serialized in parallel leveraging a
maximum possible core count of n_files*n_row_groups*n_columns.
|
| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1
| (writing) By default parallel parquet writer is tuned
for minimum memory usage in a streaming execution plan. You may see a
performance benefit when writing large parquet files by increasing
maximum_parallel_row_group_writers and
maximum_buffered_record_batches_per_stream if your system has idle cores and
can tolerate additional memory usage. Boosting these values is likely
worthwhile when writing out already in-memory data, such as from a cached data
frame.
|
| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2
| (writing) By default parallel parquet writer is tuned
for minimum memory usage in a streaming execution plan. You may see a
performance benefit when writing large parquet files by increasing
maximum_parallel_row_group_writers and
maximum_buffered_record_batches_per_stream if your system has idle cores and
can tolerate additional memory usage. Boosting these values is likely
worthwhile when writing out already in-memory data, such as from a cached data
frame.
|
+| datafusion.execution.parquet.file_decryption_properties |
NULL | Optional file decryption properties
|
Review Comment:
Yes. This is a good suggestion. We actually need to update the
`TableParquetOptions` docs and remove this entry since this got moved. @alamb
One question. Can you suggest where to put a CLI usage example? I guess I could
add something under `datafusion-cli/tests/sql`. The options will look like what
we have for KMS but I want to setup a running example. e.g. for the KMS we have:
```
let ddl = format!(
"CREATE EXTERNAL TABLE encrypted_parquet_table_2 \
STORED AS PARQUET LOCATION '{file_path}' OPTIONS (\
'format.crypto.factory_id' '{ENCRYPTION_FACTORY_ID}' \
)"
);
```
--
This is an automat
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2165219803
##
datafusion/core/src/dataframe/parquet.rs:
##
@@ -246,4 +246,72 @@ mod tests {
Ok(())
}
+
+#[tokio::test]
+async fn roundtrip_parquet_with_encryption() -> Result<()> {
Review Comment:
Maybe we can move it in a follow on PR. -- I would sort of expect the tests
to be in
https://github.com/apache/datafusion/tree/main/datafusion/core/tests/dataframe
Perhaps in a file named `parquet.rs`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2165202964
##
datafusion/core/src/dataframe/parquet.rs:
##
@@ -246,4 +246,72 @@ mod tests {
Ok(())
}
+
+#[tokio::test]
+async fn roundtrip_parquet_with_encryption() -> Result<()> {
Review Comment:
I'm happy to move it if you think we should. As you note, I am following the
pattern of the existing tests but it may fit better elsewhere.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2164809665
##
datafusion/core/src/dataframe/parquet.rs:
##
@@ -246,4 +246,72 @@ mod tests {
Ok(())
}
+
+#[tokio::test]
+async fn roundtrip_parquet_with_encryption() -> Result<()> {
Review Comment:
I wonder why this isn't in core/tests as well 🤔 (I see you are just
following the existing pattern, I just noticed this while reviewing this PR)
##
datafusion-examples/examples/parquet_encrypted.rs:
##
@@ -0,0 +1,118 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use datafusion::common::DataFusionError;
Review Comment:
I ran this example and it works great
```
===
Encrypted Parquet DataFrame:
Schema:
+++--++++++---+-++-+
| describe | id | bool_col | tinyint_col|
smallint_col | int_col| bigint_col | float_col
| double_col| date_string_col | string_col | timestamp_col |
+++--++++++---+-++-+
| count | 8.0| 8| 8.0| 8.0
| 8.0| 8.0| 8.0| 8.0
| 8 | 8 | 8 |
| null_count | 0.0| 0| 0.0| 0.0
| 0.0| 0.0| 0.0| 0.0
| 0 | 0 | 0 |
| mean | 3.5| null | 0.5| 0.5
| 0.5| 5.0| 0.55011920929 | 5.05
| null| null | null|
| std| 2.4494897427831783 | null | 0.5345224838248488 |
0.5345224838248488 | 0.5345224838248488 | 5.3452248382484875 |
0.5879747449513427 | 5.398677086630973 | null| null | null
|
| min| 0.0| null | 0.0| 0.0
| 0.0| 0.0| 0.0| 0.0
| 01/01/09| 0 | 2009-01-01T00:00:00 |
| max| 7.0| null | 1.0| 1.0
| 1.0| 10.0 | 1.10023841858 | 10.1
| 04/01/09| 1 | 2009-04-01T00:01:00 |
| median | 3.0| null | 0.0| 0.0
| 0.0| 5.0| 0.55011920929 | 5.05
| null| null | null|
+++--++++++---+-++-+
Selected rows and columns:
++--+-+
| id | bool_col | timestamp_col |
++--+-+
| 6 | true | 2009-04-01T00:00:00 |
| 7 | false| 2009-04-01T00:01:00 |
++--+-+
```
##
docs/source/user-guide/configs.md:
##
@@ -81,6 +81,8 @@ Environment variables are read during `SessionConfig`
initialisation so they mus
| datafusion.execution.parquet.allow_single_file_parallelism |
true | (writing) Controls whether DataFusion will attempt
to speed up writing parquet files by serializing them in parallel. Each column
in each row group in each output file are serialized in parallel leveraging a
maximum possible core count of n_files*
Re: [PR] feat: Parquet modular encryption [datafusion]
mbutrovich commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-3000137303 > I am sorry I haven't had a chance to review this yet. It would be great if @mbutrovich could also take a look. I have this on my list to review but I haven't been able to find the time yet I still owe this a look. I am traveling until July 7 unfortunately and likely won't get a chance to put it through its paces with Comet until after then (need to do some Comet work to get it working with this branch). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
adamreeve commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-2998703950 I've been experimenting with how this work could be extended to support more ways of configuring encryption beyond having fixed and known AES keys for all files. For example, data encryption keys are often randomly generated per file in multi-file datasets, and the keys are stored encrypted in the Parquet file's encryption metadata. I've got an example of how this could work that integrates with the [parquet-key-management](https://crates.io/crates/parquet-key-management) crate in a [draft PR here](https://github.com/corwinjoy/datafusion/pull/4/files) if anyone is interested. I've added a new `EncryptionFactory` trait for dynamically generating file encryption and decryption properties, and used a registry of these in the runtime environment to allow identifying the encryption factory with a string identifier for compatibility with string based configuration. This should be a follow up PR rather than part of this PR, but I think it's worth mentioning here as this will require adding a separate way to configure encryption rather than using the new `ConfigFileDecryptionProperties` and `ConfigFileEncryptionProperties` types in this PR. In theory, using fixed AES keys could be implemented with an `EncryptionFactory` implementation, but the configuration for this is a bit clunky and opaque, so I think it makes sense to have more direct support for this simple scenario. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2151077910
##
datafusion/common/src/config.rs:
##
@@ -591,6 +930,12 @@ config_namespace! {
/// writing out already in-memory data, such as from a cached
/// data frame.
pub maximum_buffered_record_batches_per_stream: usize, default = 2
+
+/// Optional file decryption properties
+pub file_decryption_properties:
Option, default = None
+
+/// Optional file encryption properties
+pub file_encryption_properties:
Option, default = None
Review Comment:
@adamreeve Has a nice PR to move this all to a crypto namespace which cleans
this up a lot. We are still debating a bit, since we want to understand the
impact downstream for tools like delta-rs.
https://github.com/corwinjoy/datafusion/pull/5
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
adamreeve commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2151059658
##
datafusion/common/src/config.rs:
##
@@ -188,6 +195,338 @@ macro_rules! config_namespace {
}
}
+#[derive(Clone, Default, Debug, PartialEq)]
+pub struct ConfigFileEncryptionProperties {
+pub encrypt_footer: bool, // default = false
+pub footer_key_as_hex: String,
+pub footer_key_metadata_as_hex: String,
+pub column_keys_as_hex: HashMap,
+pub column_metadata_as_hex: HashMap,
+pub aad_prefix_as_hex: String,
+pub store_aad_prefix: bool, // default = false
+}
+
+impl ConfigFileEncryptionProperties {
+/// Return new default TableParquetOptions
+pub fn new() -> Self {
+Self::default()
+}
+}
+
+impl ConfigField for ConfigFileEncryptionProperties {
+fn visit(&self, v: &mut V, key_prefix: &str, _description:
&'static str) {
+let key = format!("{key_prefix}.encrypt_footer");
+let desc = "Encrypt the footer";
+self.encrypt_footer.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.footer_key_as_hex");
+let desc = "Key to use for the parquet footer";
+self.footer_key_as_hex.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.footer_key_metadata_as_hex");
+let desc = "Metadata to use for the parquet footer";
+self.footer_key_metadata_as_hex.visit(v, key.as_str(), desc);
+
+let desc = "Per column encryption keys";
+for (col_name, col_val) in self.column_keys_as_hex.iter() {
+let key = format!("{key_prefix}.column_keys_as_hex.{col_name}");
+col_val.visit(v, key.as_str(), desc);
+}
+
+let desc = "Per column metadata";
+for (col_name, col_val) in self.column_metadata_as_hex.iter() {
+let key =
format!("{key_prefix}.column_metadata_as_hex.{col_name}");
+col_val.visit(v, key.as_str(), desc);
+}
+
+let key = format!("{key_prefix}.aad_prefix_as_hex");
+let desc = "AAD prefix to use";
+self.aad_prefix_as_hex.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.store_aad_prefix");
+let desc = "If true, store the AAD prefix";
+self.store_aad_prefix.visit(v, key.as_str(), desc);
+}
+
+fn set(&mut self, key: &str, value: &str) -> Result<()> {
+// Any hex encoded values must be pre-encoded using
+// hex::encode() before calling set.
+if key.starts_with("column_keys_as_hex.") {
+let k = match key.split(".").collect::>()[..] {
Review Comment:
I've opened a PR against your branch that implements the suggestion to move
these configuration options under a new field in `TableParquetOptions`:
https://github.com/corwinjoy/datafusion/pull/5. I think this worked quite
nicely and simplified the ConfigField implementations
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-2978284818 I am sorry I haven't had a chance to review this yet. It would be great if @mbutrovich could also take a look. I have this on my list to review but I haven't been able to find the time yet -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
mbutrovich commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-2977839997 Thank you and @adamreeve for driving so much of the modular encryption work! I'll take a look at this branch this week and see how this might get Comet supporting modular encryption within Spark, or if any obvious gaps jump out at me. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-2968354595 @alamb One piece I would like to solicit feedback on is if there is a way to leverage the existing tests to more thoroughly vet encryption. What I mean by that, is that we uncovered a read bug when using filters in a query, and I worry that there could be other edge cases that might not be covered. What I would like to do is take an encrypted parquet file and then run the datafusion SQL tests over it (and maybe other operation tests). This would help to make sure that all the SQL operations are really covered. And maybe in addition, somehow double-check things like statistics and bloom filters? Anyway, I'm hoping there is a way to leverage the existing test suite to cover these cases. Any suggestions? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
alamb commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2139135378
##
datafusion/common/src/config.rs:
##
@@ -591,6 +930,12 @@ config_namespace! {
/// writing out already in-memory data, such as from a cached
/// data frame.
pub maximum_buffered_record_batches_per_stream: usize, default = 2
+
+/// Optional file decryption properties
+pub file_decryption_properties:
Option, default = None
+
+/// Optional file encryption properties
+pub file_encryption_properties:
Option, default = None
Review Comment:
maybe @metesynnada or @berkaysynnada have some ideas of how to do this
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
adamreeve commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136730391
##
datafusion/datasource-parquet/src/file_format.rs:
##
@@ -1259,9 +1302,14 @@ impl FileSink for ParquetSink {
object_store: Arc,
) -> Result {
let parquet_opts = &self.parquet_options;
-let allow_single_file_parallelism =
+let mut allow_single_file_parallelism =
parquet_opts.global.allow_single_file_parallelism;
+if parquet_opts.global.file_encryption_properties.is_some() {
+// For now, arrow-rs does not support parallel writes with
encryption
+allow_single_file_parallelism = false;
+}
Review Comment:
There is an arrow-rs issue for this
(https://github.com/apache/arrow-rs/issues/7359). Maybe we should reference
that in a comment and add a todo note to remove this code once that's fixed and
included in version of the parquet crate being used?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136710028
##
benchmarks/src/bin/dfbench.rs:
##
@@ -60,11 +60,11 @@ pub async fn main() -> Result<()> {
Options::Cancellation(opt) => opt.run().await,
Options::Clickbench(opt) => opt.run().await,
Options::H2o(opt) => opt.run().await,
-Options::Imdb(opt) => opt.run().await,
+Options::Imdb(opt) => Box::pin(opt.run()).await,
Review Comment:
requested by clippy
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136714217
##
datafusion/common/src/config.rs:
##
@@ -188,6 +195,338 @@ macro_rules! config_namespace {
}
}
+#[derive(Clone, Default, Debug, PartialEq)]
+pub struct ConfigFileEncryptionProperties {
+pub encrypt_footer: bool, // default = false
+pub footer_key_as_hex: String,
+pub footer_key_metadata_as_hex: String,
+pub column_keys_as_hex: HashMap,
+pub column_metadata_as_hex: HashMap,
+pub aad_prefix_as_hex: String,
+pub store_aad_prefix: bool, // default = false
+}
+
+impl ConfigFileEncryptionProperties {
+/// Return new default TableParquetOptions
+pub fn new() -> Self {
+Self::default()
+}
+}
+
+impl ConfigField for ConfigFileEncryptionProperties {
+fn visit(&self, v: &mut V, key_prefix: &str, _description:
&'static str) {
+let key = format!("{key_prefix}.encrypt_footer");
+let desc = "Encrypt the footer";
+self.encrypt_footer.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.footer_key_as_hex");
+let desc = "Key to use for the parquet footer";
+self.footer_key_as_hex.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.footer_key_metadata_as_hex");
+let desc = "Metadata to use for the parquet footer";
+self.footer_key_metadata_as_hex.visit(v, key.as_str(), desc);
+
+let desc = "Per column encryption keys";
+for (col_name, col_val) in self.column_keys_as_hex.iter() {
+let key = format!("{key_prefix}.column_keys_as_hex.{col_name}");
+col_val.visit(v, key.as_str(), desc);
+}
+
+let desc = "Per column metadata";
+for (col_name, col_val) in self.column_metadata_as_hex.iter() {
+let key =
format!("{key_prefix}.column_metadata_as_hex.{col_name}");
+col_val.visit(v, key.as_str(), desc);
+}
+
+let key = format!("{key_prefix}.aad_prefix_as_hex");
+let desc = "AAD prefix to use";
+self.aad_prefix_as_hex.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.store_aad_prefix");
+let desc = "If true, store the AAD prefix";
+self.store_aad_prefix.visit(v, key.as_str(), desc);
+}
+
+fn set(&mut self, key: &str, value: &str) -> Result<()> {
+// Any hex encoded values must be pre-encoded using
+// hex::encode() before calling set.
+if key.starts_with("column_keys_as_hex.") {
+let k = match key.split(".").collect::>()[..] {
Review Comment:
We could use some feedback on how to do the column keys. Originally, I had
used a separator of '::' to match what is done with metadata fields. But
TableParquetOptions redirects all '::' delimitors as seen here.
https://github.com/corwinjoy/datafusion/blob/a81855fcbf3cfb63512c1ba124e1ebbfd5e6b15c/datafusion/common/src/config.rs#L2100
So, I'm not quite sure what to do here. For now, we use '.' to separate
columns.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on PR #16351: URL: https://github.com/apache/datafusion/pull/16351#issuecomment-2957368512 @adamreeve @rok -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136732397
##
datafusion/datasource-parquet/src/file_format.rs:
##
@@ -1259,9 +1302,14 @@ impl FileSink for ParquetSink {
object_store: Arc,
) -> Result {
let parquet_opts = &self.parquet_options;
-let allow_single_file_parallelism =
+let mut allow_single_file_parallelism =
parquet_opts.global.allow_single_file_parallelism;
+if parquet_opts.global.file_encryption_properties.is_some() {
+// For now, arrow-rs does not support parallel writes with
encryption
+allow_single_file_parallelism = false;
+}
Review Comment:
Great! I will update the code.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
adamreeve commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136733767
##
datafusion/common/src/config.rs:
##
@@ -188,6 +195,338 @@ macro_rules! config_namespace {
}
}
+#[derive(Clone, Default, Debug, PartialEq)]
+pub struct ConfigFileEncryptionProperties {
+pub encrypt_footer: bool, // default = false
+pub footer_key_as_hex: String,
+pub footer_key_metadata_as_hex: String,
+pub column_keys_as_hex: HashMap,
+pub column_metadata_as_hex: HashMap,
+pub aad_prefix_as_hex: String,
+pub store_aad_prefix: bool, // default = false
+}
+
+impl ConfigFileEncryptionProperties {
+/// Return new default TableParquetOptions
+pub fn new() -> Self {
+Self::default()
+}
+}
+
+impl ConfigField for ConfigFileEncryptionProperties {
+fn visit(&self, v: &mut V, key_prefix: &str, _description:
&'static str) {
+let key = format!("{key_prefix}.encrypt_footer");
+let desc = "Encrypt the footer";
+self.encrypt_footer.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.footer_key_as_hex");
+let desc = "Key to use for the parquet footer";
+self.footer_key_as_hex.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.footer_key_metadata_as_hex");
+let desc = "Metadata to use for the parquet footer";
+self.footer_key_metadata_as_hex.visit(v, key.as_str(), desc);
+
+let desc = "Per column encryption keys";
+for (col_name, col_val) in self.column_keys_as_hex.iter() {
+let key = format!("{key_prefix}.column_keys_as_hex.{col_name}");
+col_val.visit(v, key.as_str(), desc);
+}
+
+let desc = "Per column metadata";
+for (col_name, col_val) in self.column_metadata_as_hex.iter() {
+let key =
format!("{key_prefix}.column_metadata_as_hex.{col_name}");
+col_val.visit(v, key.as_str(), desc);
+}
+
+let key = format!("{key_prefix}.aad_prefix_as_hex");
+let desc = "AAD prefix to use";
+self.aad_prefix_as_hex.visit(v, key.as_str(), desc);
+
+let key = format!("{key_prefix}.store_aad_prefix");
+let desc = "If true, store the AAD prefix";
+self.store_aad_prefix.visit(v, key.as_str(), desc);
+}
+
+fn set(&mut self, key: &str, value: &str) -> Result<()> {
+// Any hex encoded values must be pre-encoded using
+// hex::encode() before calling set.
+if key.starts_with("column_keys_as_hex.") {
+let k = match key.split(".").collect::>()[..] {
Review Comment:
If the encryption related settings were directly set on the
`TableParquetOptions` or a `crypto`/`encryption` namespace rather than in
`ParquetOptions` then I think we could avoid this issue. But then they'd
probably need to be included in `ParquetReadOptions` too to work with
`SessionContext::read_parquet` (see related comment at
https://github.com/apache/datafusion/pull/16351/files#r2136718671).
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136718671
##
datafusion/common/src/config.rs:
##
@@ -591,6 +930,12 @@ config_namespace! {
/// writing out already in-memory data, such as from a cached
/// data frame.
pub maximum_buffered_record_batches_per_stream: usize, default = 2
+
+/// Optional file decryption properties
+pub file_decryption_properties:
Option, default = None
+
+/// Optional file encryption properties
+pub file_encryption_properties:
Option, default = None
Review Comment:
@adamreeve and I are not completely sure where these settings should go. On
the session context there's only a way to set the "global" `ParquetOptions` but
not `TableParquetOptions`, which contains extra table-specific settings.
It does feel a bit wrong to put file-specific decryption properties in the
execution context (see later examples). Eg. if users were reading two different
encrypted Parquet files in one query they might need to set different
decryption properties for each file, so setting them in the execution context
wouldn't work. At the moment I think this scenario would require creating
separate listing tables and specifying `TableParquetOptions`. That's an edge
case so maybe I'm overthinking this, but maybe being able to set file
decryption properties in `ParquetReadOptions` would be a good idea?
This doesn't really fit all that well with the reader options that Parquet
has, though.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136721651
##
datafusion/datasource-parquet/src/file_format.rs:
##
@@ -1259,9 +1302,14 @@ impl FileSink for ParquetSink {
object_store: Arc,
) -> Result {
let parquet_opts = &self.parquet_options;
-let allow_single_file_parallelism =
+let mut allow_single_file_parallelism =
parquet_opts.global.allow_single_file_parallelism;
+if parquet_opts.global.file_encryption_properties.is_some() {
+// For now, arrow-rs does not support parallel writes with
encryption
+allow_single_file_parallelism = false;
+}
Review Comment:
We noted this problem in the original PR and @rok is working on it. Would be
nice to have a tracking issue for this @adamreeve. Noted here:
https://github.com/apache/arrow-rs/pull/7111/files#r2015196618
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136718671
##
datafusion/common/src/config.rs:
##
@@ -591,6 +930,12 @@ config_namespace! {
/// writing out already in-memory data, such as from a cached
/// data frame.
pub maximum_buffered_record_batches_per_stream: usize, default = 2
+
+/// Optional file decryption properties
+pub file_decryption_properties:
Option, default = None
+
+/// Optional file encryption properties
+pub file_encryption_properties:
Option, default = None
Review Comment:
@adamreeve and I are not completely sure where this should go. On the
session context there's only a way to set the "global" `ParquetOptions` but not
`TableParquetOptions`, which contains extra table-specific settings.
It does feel a bit wrong to put file-specific decryption properties in the
execution context (see later examples). Eg. if users were reading two different
encrypted Parquet files in one query they might need to set different
decryption properties for each file, so setting them in the execution context
wouldn't work. At the moment I think this scenario would require creating
separate listing tables and specifying `TableParquetOptions`. That's an edge
case so maybe I'm overthinking this, but maybe being able to set file
decryption properties in `ParquetReadOptions` would be a good idea?
This doesn't really fit all that well with the reader options that Parquet
has, though.
##
datafusion/common/src/config.rs:
##
@@ -2231,6 +2575,130 @@ mod tests {
);
}
+#[cfg(feature = "parquet")]
+#[test]
+fn parquet_table_encryption() {
+use crate::config::{
+ConfigFileDecryptionProperties, ConfigFileEncryptionProperties,
+};
+use parquet::encryption::decrypt::FileDecryptionProperties;
+use parquet::encryption::encrypt::FileEncryptionProperties;
+
+let footer_key = b"0123456789012345".to_vec(); // 128bit/16
+let column_names = vec!["double_field", "float_field"];
+let column_keys =
+vec![b"1234567890123450".to_vec(), b"1234567890123451".to_vec()];
+
+let file_encryption_properties =
+FileEncryptionProperties::builder(footer_key.clone())
+.with_column_keys(column_names.clone(), column_keys.clone())
+.unwrap()
+.build()
+.unwrap();
+
+let decryption_properties =
FileDecryptionProperties::builder(footer_key.clone())
+.with_column_keys(column_names.clone(), column_keys.clone())
+.unwrap()
+.build()
+.unwrap();
+
+// Test round-trip
+let config_encrypt: ConfigFileEncryptionProperties =
+(&file_encryption_properties).into();
+let encryption_properties_built: FileEncryptionProperties =
+config_encrypt.clone().into();
+assert_eq!(file_encryption_properties, encryption_properties_built);
+
+let config_decrypt: ConfigFileDecryptionProperties =
+(&decryption_properties).into();
+let decryption_properties_built: FileDecryptionProperties =
+config_decrypt.clone().into();
+assert_eq!(decryption_properties, decryption_properties_built);
+
+
///
+// Test encryption config
+
+// Display original encryption config
+// println!("{:#?}", config_encrypt);
+
+let mut table_config = TableOptions::new();
+table_config.set_config_format(ConfigFileType::PARQUET);
+table_config
+.parquet
+.set(
+"file_encryption_properties.encrypt_footer",
+config_encrypt.encrypt_footer.to_string().as_str(),
+)
+.unwrap();
+table_config
+.parquet
+.set(
+"file_encryption_properties.footer_key_as_hex",
+config_encrypt.footer_key_as_hex.as_str(),
+)
+.unwrap();
+
+for (i, col_name) in column_names.iter().enumerate() {
+let key =
format!("file_encryption_properties.column_keys_as_hex.{col_name}");
Review Comment:
Note use of '.' as separator for column name, as mentioned above.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] feat: Parquet modular encryption [datafusion]
corwinjoy commented on code in PR #16351:
URL: https://github.com/apache/datafusion/pull/16351#discussion_r2136715685
##
datafusion/common/src/config.rs:
##
@@ -188,6 +195,338 @@ macro_rules! config_namespace {
}
}
+#[derive(Clone, Default, Debug, PartialEq)]
+pub struct ConfigFileEncryptionProperties {
+pub encrypt_footer: bool, // default = false
+pub footer_key_as_hex: String,
+pub footer_key_metadata_as_hex: String,
+pub column_keys_as_hex: HashMap,
+pub column_metadata_as_hex: HashMap,
+pub aad_prefix_as_hex: String,
+pub store_aad_prefix: bool, // default = false
+}
Review Comment:
We create a separate Config struct, then use From methods to convert back
and forth from the underlying parquet `FileEncryptionProperties`.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
