|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.apache.comet.parquet; |
| 21 | + |
| 22 | +import java.util.concurrent.ConcurrentHashMap; |
| 23 | + |
| 24 | +import org.apache.hadoop.conf.Configuration; |
| 25 | +import org.apache.hadoop.fs.Path; |
| 26 | +import org.apache.parquet.crypto.DecryptionKeyRetriever; |
| 27 | +import org.apache.parquet.crypto.DecryptionPropertiesFactory; |
| 28 | +import org.apache.parquet.crypto.FileDecryptionProperties; |
| 29 | +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; |
| 30 | + |
| 31 | +// spotless:off |
| 32 | +/* |
| 33 | + * Architecture Overview: |
| 34 | + * |
| 35 | + * JVM Side | Native Side |
| 36 | + * ┌─────────────────────────────────────┐ | ┌─────────────────────────────────────┐ |
| 37 | + * │ CometFileKeyUnwrapper │ | │ Parquet File Reading │ |
| 38 | + * │ │ | │ │ |
| 39 | + * │ ┌─────────────────────────────┐ │ | │ ┌─────────────────────────────┐ │ |
| 40 | + * │ │ hadoopConf │ │ | │ │ file1.parquet │ │ |
| 41 | + * │ │ (Configuration) │ │ | │ │ file2.parquet │ │ |
| 42 | + * │ └─────────────────────────────┘ │ | │ │ file3.parquet │ │ |
| 43 | + * │ │ │ | │ └─────────────────────────────┘ │ |
| 44 | + * │ ▼ │ | │ │ │ |
| 45 | + * │ ┌─────────────────────────────┐ │ | │ │ │ |
| 46 | + * │ │ factoryCache │ │ | │ ▼ │ |
| 47 | + * │ │ (many-to-one mapping) │ │ | │ ┌─────────────────────────────┐ │ |
| 48 | + * │ │ │ │ | │ │ Parse file metadata & │ │ |
| 49 | + * │ │ file1 ──┐ │ │ | │ │ extract keyMetadata │ │ |
| 50 | + * │ │ file2 ──┼─► DecryptionProps │ │ | │ └─────────────────────────────┘ │ |
| 51 | + * │ │ file3 ──┘ Factory │ │ | │ │ │ |
| 52 | + * │ └─────────────────────────────┘ │ | │ │ │ |
| 53 | + * │ │ │ | │ ▼ │ |
| 54 | + * │ ▼ │ | │ ╔═════════════════════════════╗ │ |
| 55 | + * │ ┌─────────────────────────────┐ │ | │ ║ JNI CALL: ║ │ |
| 56 | + * │ │ retrieverCache │ │ | │ ║ getKey(filePath, ║ │ |
| 57 | + * │ │ filePath -> KeyRetriever │◄───┼───┼───┼──║ keyMetadata) ║ │ |
| 58 | + * │ └─────────────────────────────┘ │ | │ ╚═════════════════════════════╝ │ |
| 59 | + * │ │ │ | │ │ |
| 60 | + * │ ▼ │ | │ │ |
| 61 | + * │ ┌─────────────────────────────┐ │ | │ │ |
| 62 | + * │ │ DecryptionKeyRetriever │ │ | │ │ |
| 63 | + * │ │ .getKey(keyMetadata) │ │ | │ │ |
| 64 | + * │ └─────────────────────────────┘ │ | │ │ |
| 65 | + * │ │ │ | │ │ |
| 66 | + * │ ▼ │ | │ │ |
| 67 | + * │ ┌─────────────────────────────┐ │ | │ ┌─────────────────────────────┐ │ |
| 68 | + * │ │ return key bytes │────┼───┼───┼─►│ Use key for decryption │ │ |
| 69 | + * │ └─────────────────────────────┘ │ | │ │ of parquet data │ │ |
| 70 | + * └─────────────────────────────────────┘ | │ └─────────────────────────────┘ │ |
| 71 | + * | └─────────────────────────────────────┘ |
| 72 | + * | |
| 73 | + * JNI Boundary |
| 74 | + * |
| 75 | + * Setup Phase (storeDecryptionKeyRetriever): |
| 76 | + * 1. hadoopConf → DecryptionPropertiesFactory (cached in factoryCache) |
| 77 | + * 2. Factory + filePath → DecryptionKeyRetriever (cached in retrieverCache) |
| 78 | + * |
| 79 | + * Runtime Phase (getKey): |
| 80 | + * 3. Native code calls getKey(filePath, keyMetadata) ──► JVM |
| 81 | + * 4. Retrieve cached DecryptionKeyRetriever for filePath |
| 82 | + * 5. KeyRetriever.getKey(keyMetadata) → decrypted key bytes |
| 83 | + * 6. Return key bytes ──► Native code for parquet decryption |
| 84 | + */ |
| 85 | +// spotless:on |
| 86 | + |
| 87 | +/** |
| 88 | + * Helper class to access DecryptionKeyRetriever.getKey from native code via JNI. This class handles |
| 89 | + * the complexity of creating and caching properly configured DecryptionKeyRetriever instances using |
| 90 | + * DecryptionPropertiesFactory. The life of this object is meant to map to a single Comet plan, so |
| 91 | + * associated with CometExecIterator. |
| 92 | + */ |
| 93 | +public class CometFileKeyUnwrapper { |
| 94 | + |
| 95 | + // Each file path gets a unique DecryptionKeyRetriever |
| 96 | + private final ConcurrentHashMap<String, DecryptionKeyRetriever> retrieverCache = |
| 97 | + new ConcurrentHashMap<>(); |
| 98 | + |
| 99 | + // Cache the factory since we should be using the same hadoopConf for every file in this scan. |
| 100 | + private DecryptionPropertiesFactory factory = null; |
| 101 | + // Cache the hadoopConf just to assert the assumption above. |
| 102 | + private Configuration conf = null; |
| 103 | + |
| 104 | + /** |
| 105 | + * Creates and stores a DecryptionKeyRetriever instance for the given file path. |
| 106 | + * |
| 107 | + * @param filePath The path to the Parquet file |
| 108 | + * @param hadoopConf The Hadoop Configuration to use for this file path |
| 109 | + */ |
| 110 | + public void storeDecryptionKeyRetriever(final String filePath, final Configuration hadoopConf) { |
| 111 | + // Use DecryptionPropertiesFactory.loadFactory to get the factory and then call |
| 112 | + // getFileDecryptionProperties |
| 113 | + if (factory == null) { |
| 114 | + factory = DecryptionPropertiesFactory.loadFactory(hadoopConf); |
| 115 | + conf = hadoopConf; |
| 116 | + } else { |
| 117 | + // Check the assumption that all files have the same hadoopConf and thus same Factory |
| 118 | + assert (conf == hadoopConf); |
| 119 | + } |
| 120 | + Path path = new Path(filePath); |
| 121 | + FileDecryptionProperties decryptionProperties = |
| 122 | + factory.getFileDecryptionProperties(hadoopConf, path); |
| 123 | + |
| 124 | + DecryptionKeyRetriever keyRetriever = decryptionProperties.getKeyRetriever(); |
| 125 | + retrieverCache.put(filePath, keyRetriever); |
| 126 | + } |
| 127 | + |
| 128 | + /** |
| 129 | + * Gets the decryption key for the given key metadata using the cached DecryptionKeyRetriever for |
| 130 | + * the specified file path. |
| 131 | + * |
| 132 | + * @param filePath The path to the Parquet file |
| 133 | + * @param keyMetadata The key metadata bytes from the Parquet file |
| 134 | + * @return The decrypted key bytes |
| 135 | + * @throws ParquetCryptoRuntimeException if key unwrapping fails |
| 136 | + */ |
| 137 | + public byte[] getKey(final String filePath, final byte[] keyMetadata) |
| 138 | + throws ParquetCryptoRuntimeException { |
| 139 | + DecryptionKeyRetriever keyRetriever = retrieverCache.get(filePath); |
| 140 | + if (keyRetriever == null) { |
| 141 | + throw new ParquetCryptoRuntimeException( |
| 142 | + "Failed to find DecryptionKeyRetriever for path: " + filePath); |
| 143 | + } |
| 144 | + return keyRetriever.getKey(keyMetadata); |
| 145 | + } |
| 146 | +} |
0 commit comments