mbutrovich commented on code in PR #2447:
URL: https://github.com/apache/datafusion-comet/pull/2447#discussion_r2391052576


##########
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 =

Review Comment:
   Session hadoopConf is not what the scans use though. They add all the 
relation options (Parquet options like keys) to the hadoopConf, so each scan 
can have a unique hadoopConf. Whether we could have a Comet plan with multiple 
Parquet scans is the real question.



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