adamreeve commented on code in PR #9203:
URL: https://github.com/apache/arrow-rs/pull/9203#discussion_r3056397795


##########
parquet/tests/encryption/encryption_async.rs:
##########
@@ -183,113 +323,285 @@ async fn 
test_plaintext_footer_read_without_decryption() {
 
 #[tokio::test]
 async fn test_non_uniform_encryption() {
-    let test_data = arrow::util::test_util::parquet_test_data();
-    let path = 
format!("{test_data}/encrypt_columns_and_footer.parquet.encrypted");
-    let mut file = File::open(&path).await.unwrap();
+    async fn non_uniform_encryption(footer_key: &[u8], column_keys: &[(&str, 
&[u8])]) {
+        let path = encryption_util::encrypted_data_path(
+            footer_key,
+            "encrypt_columns_and_footer.parquet.encrypted",
+        );
+        let mut file = File::open(&path).await.unwrap();
 
-    let footer_key = "0123456789012345".as_bytes().to_vec(); // 128bit/16
-    let column_1_key = "1234567890123450".as_bytes().to_vec();
-    let column_2_key = "1234567890123451".as_bytes().to_vec();
+        let mut builder = 
FileDecryptionProperties::builder(footer_key.to_vec());
+        for (column_name, key) in column_keys {
+            builder = builder.with_column_key(column_name, key.to_vec());
+        }
+        let decryption_properties = builder.build().unwrap();
 
-    let decryption_properties = 
FileDecryptionProperties::builder(footer_key.to_vec())
-        .with_column_key("double_field", column_1_key)
-        .with_column_key("float_field", column_2_key)
-        .build()
-        .unwrap();
+        verify_encryption_test_file_read_async(&mut file, 
decryption_properties)
+            .await
+            .unwrap();
+    }
 
-    verify_encryption_test_file_read_async(&mut file, decryption_properties)
-        .await
-        .unwrap();
+    // AES-128
+    non_uniform_encryption(
+        b"0123456789012345",
+        &[
+            ("double_field", b"1234567890123450".as_slice()),
+            ("float_field", b"1234567890123451".as_slice()),
+        ],
+    )
+    .await;
+
+    // AES-256
+    non_uniform_encryption(
+        b"01234567890123456789012345678901", // 256bit/32
+        &[
+            (
+                "double_field",
+                b"12345678901234567890123456789012".as_slice(),
+            ),
+            (
+                "float_field",
+                b"12345678901234567890123456789013".as_slice(),
+            ),
+            (
+                "boolean_field",
+                b"12345678901234567890123456789014".as_slice(),
+            ),
+            (
+                "int32_field",
+                b"12345678901234567890123456789015".as_slice(),
+            ),
+            ("ba_field", b"12345678901234567890123456789016".as_slice()),
+            ("flba_field", b"12345678901234567890123456789017".as_slice()),
+            (
+                "int64_field",
+                b"12345678901234567890123456789018".as_slice(),
+            ),
+            (
+                "int96_field",
+                b"12345678901234567890123456789019".as_slice(),
+            ),
+        ],
+    )
+    .await;
 }
 
 #[tokio::test]
 async fn test_uniform_encryption() {
-    let test_data = arrow::util::test_util::parquet_test_data();
-    let path = format!("{test_data}/uniform_encryption.parquet.encrypted");
-    let mut file = File::open(&path).await.unwrap();
+    async fn uniform_encryption(footer_key: &[u8], column_keys: &[(&str, 
&[u8])]) {
+        let path = encryption_util::encrypted_data_path(
+            footer_key,
+            "uniform_encryption.parquet.encrypted",
+        );
+        let mut file = File::open(&path).await.unwrap();
 
-    let key_code: &[u8] = "0123456789012345".as_bytes();
-    let decryption_properties = 
FileDecryptionProperties::builder(key_code.to_vec())
-        .build()
-        .unwrap();
+        let mut builder = 
FileDecryptionProperties::builder(footer_key.to_vec());
+        for (column_name, key) in column_keys {
+            builder = builder.with_column_key(column_name, key.to_vec());
+        }
+        let decryption_properties = builder.build().unwrap();
 
-    verify_encryption_test_file_read_async(&mut file, decryption_properties)
-        .await
-        .unwrap();
+        verify_encryption_test_file_read_async(&mut file, 
decryption_properties)
+            .await
+            .unwrap();
+    }
+
+    // AES-128: there is always a footer key even with a plaintext footer,
+    // but this is used for signing the footer.
+    uniform_encryption(
+        b"0123456789012345", // 128bit/16
+        &[],
+    )
+    .await;
+
+    // AES-256
+    uniform_encryption(
+        b"01234567890123456789012345678901", // 256bit/32
+        &[],
+    )
+    .await;
 }
 
 #[tokio::test]
 async fn test_aes_ctr_encryption() {
-    let test_data = arrow::util::test_util::parquet_test_data();
-    let path = 
format!("{test_data}/encrypt_columns_and_footer_ctr.parquet.encrypted");
-    let mut file = File::open(&path).await.unwrap();
+    async fn aes_ctr_encryption(footer_key: &[u8], column_keys: &[(&str, 
&[u8])]) {
+        let path = encryption_util::encrypted_data_path(
+            footer_key,
+            "encrypt_columns_and_footer_ctr.parquet.encrypted",
+        );
+        let mut file = File::open(&path).await.unwrap();
 
-    let footer_key = "0123456789012345".as_bytes().to_vec();
-    let column_1_key = "1234567890123450".as_bytes().to_vec();
-    //let column_2_key = "1234567890123451".as_bytes().to_vec();
+        let mut builder = 
FileDecryptionProperties::builder(footer_key.to_vec());
+        for (column_name, key) in column_keys {
+            builder = builder.with_column_key(column_name, key.to_vec());
+        }
+        let decryption_properties = builder.build().unwrap();
 
-    let decryption_properties = FileDecryptionProperties::builder(footer_key)
-        .with_column_key("double_field", column_1_key.clone())
-        .with_column_key("float_field", column_1_key)
-        .build()
-        .unwrap();
+        let options =
+            
ArrowReaderOptions::new().with_file_decryption_properties(decryption_properties);
+        let metadata = ArrowReaderMetadata::load_async(&mut file, 
options).await;
 
-    let options = 
ArrowReaderOptions::new().with_file_decryption_properties(decryption_properties);
-    let metadata = ArrowReaderMetadata::load_async(&mut file, options).await;
+        match metadata {
+            Err(ParquetError::NYI(s)) => {
+                assert!(s.contains("AES_GCM_CTR_V1"));
+            }
+            _ => {
+                panic!("Expected ParquetError::NYI");
+            }
+        };
+    }
 
-    match metadata {
-        Err(ParquetError::NYI(s)) => {
-            assert!(s.contains("AES_GCM_CTR_V1"));
-        }
-        _ => {
-            panic!("Expected ParquetError::NYI");
-        }
-    };
+    // AES-128
+    aes_ctr_encryption(
+        b"0123456789012345", // 128bit/16
+        &[
+            ("double_field", b"1234567890123450".as_slice()),
+            ("float_field", b"1234567890123451".as_slice()),
+        ],
+    )
+    .await;
+
+    // AES-256
+    aes_ctr_encryption(
+        b"01234567890123456789012345678901", // 256bit/32
+        &[
+            (
+                "double_field",
+                b"12345678901234567890123456789012".as_slice(),
+            ),
+            (
+                "float_field",
+                b"12345678901234567890123456789013".as_slice(),
+            ),
+        ],
+    )
+    .await;
 }
 
 #[tokio::test]
 async fn test_decrypting_without_decryption_properties_fails() {
     let test_data = arrow::util::test_util::parquet_test_data();
-    let path = format!("{test_data}/uniform_encryption.parquet.encrypted");
-    let mut file = File::open(&path).await.unwrap();
+    let paths = [
+        format!("{test_data}/uniform_encryption.parquet.encrypted"),
+        format!("{test_data}/aes256/uniform_encryption.parquet.encrypted"),
+    ];
 
-    let options = ArrowReaderOptions::new();
-    let result = ArrowReaderMetadata::load_async(&mut file, options).await;
-    assert!(result.is_err());
-    assert_eq!(
-        result.unwrap_err().to_string(),
-        "Parquet error: Parquet file has an encrypted footer but decryption 
properties were not provided"
-    );
+    for path in &paths {
+        let mut file = File::open(&path).await.unwrap();
+
+        let options = ArrowReaderOptions::new();
+        let result = ArrowReaderMetadata::load_async(&mut file, options).await;
+        assert!(result.is_err());
+        assert_eq!(
+            result.unwrap_err().to_string(),
+            "Parquet error: Parquet file has an encrypted footer but 
decryption properties were not provided"
+        );
+    }
 }
 
 #[tokio::test]
 async fn test_write_non_uniform_encryption() {
-    let testdata = arrow::util::test_util::parquet_test_data();
-    let path = 
format!("{testdata}/encrypt_columns_and_footer.parquet.encrypted");
+    async fn write_non_uniform_encryption(
+        footer_key: &[u8],
+        column_names: Vec<&str>,
+        column_keys: Vec<Vec<u8>>,
+        encryption_column_keys: &[(&str, &[u8])],
+    ) {
+        let path = encryption_util::encrypted_data_path(
+            footer_key,
+            "encrypt_columns_and_footer.parquet.encrypted",
+        );
+
+        let decryption_properties = 
FileDecryptionProperties::builder(footer_key.to_vec())
+            .with_column_keys(column_names.to_vec(), column_keys.clone())
+            .unwrap()
+            .build()
+            .unwrap();
 
-    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 mut builder = 
FileEncryptionProperties::builder(footer_key.to_vec());
+        for (column_name, key) in encryption_column_keys {
+            builder = builder.with_column_key(column_name, key.to_vec());
+        }
+        let file_encryption_properties = builder.build().unwrap();
 
-    let decryption_properties = 
FileDecryptionProperties::builder(footer_key.clone())
-        .with_column_keys(column_names.clone(), column_keys.clone())
-        .unwrap()
-        .build()
+        read_and_roundtrip_to_encrypted_file_async(
+            &path,
+            decryption_properties,
+            file_encryption_properties,
+        )
+        .await
         .unwrap();
+    }
 
-    let file_encryption_properties = 
FileEncryptionProperties::builder(footer_key)
-        .with_column_keys(column_names, column_keys)
-        .unwrap()
-        .build()
-        .unwrap();
+    write_non_uniform_encryption(
+        b"0123456789012345", // 128bit/16,
+        vec!["double_field", "float_field"],
+        vec![b"1234567890123450".to_vec(), b"1234567890123451".to_vec()],
+        &[
+            ("double_field", b"1234567890123450".as_slice()),
+            ("float_field", b"1234567890123451".as_slice()),
+        ],
+    )
+    .await;
 
-    read_and_roundtrip_to_encrypted_file_async(
-        &path,
-        decryption_properties,
-        file_encryption_properties,
+    // AES-256
+    // The asymmetric column names is because we check column paths in the 
validate_encrypted_column_names function of [encryption::encrypt]

Review Comment:
   I looked closer into why the different "int64_field" and 
"int64_field.list.int64_field" column names are needed in the tests. This is 
mainly because the test file doesn't use the Parquet list logical type, but 
just has a top-level schema node named "int64_field" that's a repeated field. 
So it's schema path is simply "int64_field".
   
   When reading this data as Arrow, the "int64_field" name gets used when 
creating the Arrow schema, which ends up with an "int64_field" list typed 
field, and the element/item field name is also "int64_field".
   
   When writing the Arrow data back to Parquet, arrow-rs does use the 3-level 
Parquet list logical type, but rather than using the standard "element" name, 
it copies the "int64_field" name from the Arrow schema.
   
   So to read the original example file, you need an "int64_field" column key 
specified, but to write the file with arrow-rs, and also to read it back after 
it's re-written, you need a key for "int64_field.list.int64_field".
   
   I don't think there's any problem with the way the encryption code works, 
but arrow-rs using a non-standard naming scheme for the Parquet schema nodes 
might be something that should be fixed.



-- 
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]

Reply via email to