martin-g commented on code in PR #2447:
URL: https://github.com/apache/datafusion-comet/pull/2447#discussion_r2393772647


##########
common/src/main/java/org/apache/comet/parquet/CometFileKeyUnwrapper.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.
+ */
+
+package org.apache.comet.parquet;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.DecryptionKeyRetriever;
+import org.apache.parquet.crypto.DecryptionPropertiesFactory;
+import org.apache.parquet.crypto.FileDecryptionProperties;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+
+// spotless:off
+/*
+ * Architecture Overview:
+ *
+ *          JVM Side                           |                     Native 
Side
+ *   ┌─────────────────────────────────────┐   |   
┌─────────────────────────────────────┐
+ *   │     CometFileKeyUnwrapper           │   |   │       Parquet File 
Reading          │
+ *   │                                     │   |   │                           
          │
+ *   │  ┌─────────────────────────────┐    │   |   │  
┌─────────────────────────────┐    │
+ *   │  │      hadoopConf             │    │   |   │  │     file1.parquet      
     │    │
+ *   │  │   (Configuration)           │    │   |   │  │     file2.parquet      
     │    │
+ *   │  └─────────────────────────────┘    │   |   │  │     file3.parquet      
     │    │
+ *   │              │                      │   |   │  
└─────────────────────────────┘    │
+ *   │              ▼                      │   |   │              │            
          │
+ *   │  ┌─────────────────────────────┐    │   |   │              │            
          │
+ *   │  │      factoryCache           │    │   |   │              ▼            
          │
+ *   │  │   (many-to-one mapping)     │    │   |   │  
┌─────────────────────────────┐    │
+ *   │  │                             │    │   |   │  │  Parse file metadata & 
     │    │
+ *   │  │ file1 ──┐                   │    │   |   │  │  extract keyMetadata   
     │    │
+ *   │  │ file2 ──┼─► DecryptionProps │    │   |   │  
└─────────────────────────────┘    │
+ *   │  │ file3 ──┘      Factory      │    │   |   │              │            
          │
+ *   │  └─────────────────────────────┘    │   |   │              │            
          │
+ *   │              │                      │   |   │              ▼            
          │
+ *   │              ▼                      │   |   │  
╔═════════════════════════════╗    │
+ *   │  ┌─────────────────────────────┐    │   |   │  ║        JNI CALL:       
     ║    │
+ *   │  │      retrieverCache         │    │   |   │  ║       getKey(filePath, 
     ║    │
+ *   │  │  filePath -> KeyRetriever   │◄───┼───┼───┼──║        keyMetadata)    
     ║    │
+ *   │  └─────────────────────────────┘    │   |   │  
╚═════════════════════════════╝    │
+ *   │              │                      │   |   │                           
          │
+ *   │              ▼                      │   |   │                           
          │
+ *   │  ┌─────────────────────────────┐    │   |   │                           
          │
+ *   │  │  DecryptionKeyRetriever     │    │   |   │                           
          │
+ *   │  │     .getKey(keyMetadata)    │    │   |   │                           
          │
+ *   │  └─────────────────────────────┘    │   |   │                           
          │
+ *   │              │                      │   |   │                           
          │
+ *   │              ▼                      │   |   │                           
          │
+ *   │  ┌─────────────────────────────┐    │   |   │  
┌─────────────────────────────┐    │
+ *   │  │      return key bytes       │────┼───┼───┼─►│   Use key for 
decryption    │    │
+ *   │  └─────────────────────────────┘    │   |   │  │    of parquet data     
     │    │
+ *   └─────────────────────────────────────┘   |   │  
└─────────────────────────────┘    │
+ *                                             |   
└─────────────────────────────────────┘
+ *                                             |
+ *                                    JNI Boundary
+ *
+ * Setup Phase (storeDecryptionKeyRetriever):
+ * 1. hadoopConf → DecryptionPropertiesFactory (cached in factoryCache)
+ * 2. Factory + filePath → DecryptionKeyRetriever (cached in retrieverCache)
+ *
+ * Runtime Phase (getKey):
+ * 3. Native code calls getKey(filePath, keyMetadata) ──► JVM
+ * 4. Retrieve cached DecryptionKeyRetriever for filePath
+ * 5. KeyRetriever.getKey(keyMetadata) → decrypted key bytes
+ * 6. Return key bytes ──► Native code for parquet decryption
+ */
+// spotless:on
+
+/**
+ * Helper class to access DecryptionKeyRetriever.getKey from native code via 
JNI. This class handles
+ * the complexity of creating and caching properly configured 
DecryptionKeyRetriever instances using
+ * DecryptionPropertiesFactory. The life of this object is meant to map to a 
single Comet plan, so
+ * associated with CometExecIterator.
+ */
+public class CometFileKeyUnwrapper {
+
+  // Each file path gets a unique DecryptionKeyRetriever
+  private final ConcurrentHashMap<String, DecryptionKeyRetriever> 
retrieverCache =
+      new ConcurrentHashMap<>();
+
+  // Each hadoopConf yields a unique DecryptionPropertiesFactory. While it's 
unlikely that
+  // this Comet plan contains more than one hadoopConf, we don't want to 
assume that. So we'll
+  // provide the ability to cache more than one Factory with a map.
+  private final ConcurrentHashMap<Configuration, DecryptionPropertiesFactory> 
factoryCache =
+      new ConcurrentHashMap<>();
+
+  /**
+   * Creates and stores a DecryptionKeyRetriever instance for the given file 
path.
+   *
+   * @param filePath The path to the Parquet file
+   * @param hadoopConf The Hadoop Configuration to use for this file path
+   */
+  public void storeDecryptionKeyRetriever(final String filePath, final 
Configuration hadoopConf) {
+    // Use DecryptionPropertiesFactory.loadFactory to get the factory and then 
call
+    // getFileDecryptionProperties
+    DecryptionPropertiesFactory factory = factoryCache.get(hadoopConf);
+    if (factory == null) {
+      factory = DecryptionPropertiesFactory.loadFactory(hadoopConf);
+      factoryCache.put(hadoopConf, factory);

Review Comment:
   Since you use `ConcurrentMap` you probably want to use its 
`#computeIfAbsent()` method instead of `#get() + check for null + #put()`



##########
native/core/src/parquet/encryption_support.rs:
##########
@@ -0,0 +1,151 @@
+// 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 crate::execution::operators::ExecutionError;
+use crate::jvm_bridge::JVMClasses;
+use arrow::datatypes::SchemaRef;
+use async_trait::async_trait;
+use datafusion::common::extensions_options;
+use datafusion::config::EncryptionFactoryOptions;
+use datafusion::error::DataFusionError;
+use datafusion::execution::parquet_encryption::EncryptionFactory;
+use jni::objects::{GlobalRef, JMethodID};
+use object_store::path::Path;
+use parquet::encryption::decrypt::{FileDecryptionProperties, KeyRetriever};
+use parquet::encryption::encrypt::FileEncryptionProperties;
+use std::sync::Arc;
+
+pub const ENCRYPTION_FACTORY_ID: &str = "comet.jni_kms_encryption";
+
+// Options used to configure our example encryption factory
+extensions_options! {
+    pub struct CometEncryptionConfig {
+        pub url_base: String, default = "file:///".into()
+    }
+}
+
+#[derive(Debug)]
+pub struct CometEncryptionFactory {
+    pub(crate) key_unwrapper: GlobalRef,
+}
+
+/// `EncryptionFactory` is a DataFusion trait for types that generate
+/// file encryption and decryption properties.
+#[async_trait]
+impl EncryptionFactory for CometEncryptionFactory {
+    async fn get_file_encryption_properties(
+        &self,
+        _options: &EncryptionFactoryOptions,
+        _schema: &SchemaRef,
+        _file_path: &Path,
+    ) -> Result<Option<FileEncryptionProperties>, DataFusionError> {
+        Err(DataFusionError::NotImplemented(
+            "Comet does not support Parquet encryption yet."
+                .parse()
+                .unwrap(),
+        ))
+    }
+
+    /// Generate file decryption properties to use when reading a Parquet file.
+    /// Rather than provide the AES keys directly for decryption, we set a 
`KeyRetriever`
+    /// that can determine the keys using the encryption metadata.
+    async fn get_file_decryption_properties(
+        &self,
+        options: &EncryptionFactoryOptions,
+        file_path: &Path,
+    ) -> Result<Option<FileDecryptionProperties>, DataFusionError> {
+        let config: CometEncryptionConfig = options.to_extension_options()?;
+
+        let full_path: String = config.url_base + file_path.as_ref();
+        let key_retriever = CometKeyRetriever::new(&full_path, 
self.key_unwrapper.clone())
+            .map_err(|e| DataFusionError::External(Box::new(e)))?;
+        let decryption_properties =
+            
FileDecryptionProperties::with_key_retriever(Arc::new(key_retriever)).build()?;
+        Ok(Some(decryption_properties))
+    }
+}
+
+pub struct CometKeyRetriever {
+    file_path: String,
+    key_unwrapper: GlobalRef,
+    get_key_method_id: JMethodID,
+}
+
+impl CometKeyRetriever {
+    pub fn new(file_path: &str, key_unwrapper: GlobalRef) -> Result<Self, 
ExecutionError> {
+        // Get JNI environment
+        let mut env = JVMClasses::get_env()?;
+
+        Ok(CometKeyRetriever {
+            file_path: file_path.to_string(),
+            key_unwrapper,
+            get_key_method_id: env
+                .get_method_id(
+                    "org/apache/comet/parquet/CometFileKeyUnwrapper",
+                    "getKey",
+                    "(Ljava/lang/String;[B)[B",
+                )
+                .unwrap(),
+        })
+    }
+}
+
+impl KeyRetriever for CometKeyRetriever {
+    /// Get a data encryption key using the metadata stored in the Parquet 
file.
+    fn retrieve_key(&self, key_metadata: &[u8]) -> 
datafusion::parquet::errors::Result<Vec<u8>> {
+        use jni::{objects::JObject, signature::ReturnType};
+
+        // Get JNI environment
+        let mut env = JVMClasses::get_env()
+            .map_err(|e| 
datafusion::parquet::errors::ParquetError::General(e.to_string()))?;
+
+        // Get the key unwrapper instance from GlobalRef
+        let unwrapper_instance = self.key_unwrapper.as_obj();
+
+        let instance: JObject = unsafe { 
JObject::from_raw(unwrapper_instance.as_raw()) };
+
+        // Convert file path to JString
+        let file_path_jstring = env.new_string(&self.file_path).unwrap();
+
+        // Convert key_metadata to JByteArray
+        let key_metadata_array = 
env.byte_array_from_slice(key_metadata).unwrap();
+
+        // Call instance method FileKeyUnwrapper.getKey(String, byte[]) -> 
byte[]
+        let result = unsafe {
+            env.call_method_unchecked(
+                instance,
+                self.get_key_method_id,
+                ReturnType::Array,
+                &[
+                    jni::objects::JValue::from(&file_path_jstring).as_jni(),
+                    jni::objects::JValue::from(&key_metadata_array).as_jni(),
+                ],
+            )
+        };
+
+        let result = result.unwrap();

Review Comment:
   Can you use `result?` instead ? To return an Err instead of panic-ing.
   Or use pattern match and return a custom error.
   
   Few more unwraps below.



##########
native/core/src/parquet/encryption_support.rs:
##########
@@ -0,0 +1,151 @@
+// 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 crate::execution::operators::ExecutionError;
+use crate::jvm_bridge::JVMClasses;
+use arrow::datatypes::SchemaRef;
+use async_trait::async_trait;
+use datafusion::common::extensions_options;
+use datafusion::config::EncryptionFactoryOptions;
+use datafusion::error::DataFusionError;
+use datafusion::execution::parquet_encryption::EncryptionFactory;
+use jni::objects::{GlobalRef, JMethodID};
+use object_store::path::Path;
+use parquet::encryption::decrypt::{FileDecryptionProperties, KeyRetriever};
+use parquet::encryption::encrypt::FileEncryptionProperties;
+use std::sync::Arc;
+
+pub const ENCRYPTION_FACTORY_ID: &str = "comet.jni_kms_encryption";
+
+// Options used to configure our example encryption factory
+extensions_options! {
+    pub struct CometEncryptionConfig {
+        pub url_base: String, default = "file:///".into()
+    }
+}
+
+#[derive(Debug)]
+pub struct CometEncryptionFactory {
+    pub(crate) key_unwrapper: GlobalRef,
+}
+
+/// `EncryptionFactory` is a DataFusion trait for types that generate
+/// file encryption and decryption properties.
+#[async_trait]
+impl EncryptionFactory for CometEncryptionFactory {
+    async fn get_file_encryption_properties(
+        &self,
+        _options: &EncryptionFactoryOptions,
+        _schema: &SchemaRef,
+        _file_path: &Path,
+    ) -> Result<Option<FileEncryptionProperties>, DataFusionError> {
+        Err(DataFusionError::NotImplemented(
+            "Comet does not support Parquet encryption yet."
+                .parse()
+                .unwrap(),
+        ))
+    }
+
+    /// Generate file decryption properties to use when reading a Parquet file.
+    /// Rather than provide the AES keys directly for decryption, we set a 
`KeyRetriever`
+    /// that can determine the keys using the encryption metadata.
+    async fn get_file_decryption_properties(
+        &self,
+        options: &EncryptionFactoryOptions,
+        file_path: &Path,
+    ) -> Result<Option<FileDecryptionProperties>, DataFusionError> {
+        let config: CometEncryptionConfig = options.to_extension_options()?;
+
+        let full_path: String = config.url_base + file_path.as_ref();
+        let key_retriever = CometKeyRetriever::new(&full_path, 
self.key_unwrapper.clone())
+            .map_err(|e| DataFusionError::External(Box::new(e)))?;
+        let decryption_properties =
+            
FileDecryptionProperties::with_key_retriever(Arc::new(key_retriever)).build()?;
+        Ok(Some(decryption_properties))
+    }
+}
+
+pub struct CometKeyRetriever {
+    file_path: String,
+    key_unwrapper: GlobalRef,
+    get_key_method_id: JMethodID,
+}
+
+impl CometKeyRetriever {
+    pub fn new(file_path: &str, key_unwrapper: GlobalRef) -> Result<Self, 
ExecutionError> {
+        // Get JNI environment
+        let mut env = JVMClasses::get_env()?;

Review Comment:
   Below you use
   ```rust
   let mut env = JVMClasses::get_env()
               .map_err(|e| 
datafusion::parquet::errors::ParquetError::General(e.to_string()))?;
   ```
   No need of error mapping 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]

Reply via email to