parthchandra commented on code in PR #2447: URL: https://github.com/apache/datafusion-comet/pull/2447#discussion_r2388718794
########## 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: There is only one hadoop conf in a spark session so this may be overkill. ########## common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java: ########## @@ -410,6 +410,14 @@ public void init() throws Throwable { } } + boolean encryptionEnabled = CometParquetUtils.encryptionEnabled(conf); + + // Create keyUnwrapper if encryption is enabled + CometFileKeyUnwrapper keyUnwrapper = encryptionEnabled ? new CometFileKeyUnwrapper() : null; + if (encryptionEnabled) { + keyUnwrapper.storeDecryptionKeyRetriever(file.filePath().toString(), conf); + } + Review Comment: ```suggestion CometFileKeyUnwrapper keyUnwrapper = null; if (encryptionEnabled) { keyUnwrapper = new CometFileKeyUnwrapper; keyUnwrapper.storeDecryptionKeyRetriever(file.filePath().toString(), conf); } ``` ########## 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 = Review Comment: Every file path? This can get rather large when the number of files starts to reach 100K or more. ########## common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala: ########## @@ -20,13 +20,30 @@ package org.apache.comet.parquet import org.apache.hadoop.conf.Configuration +import org.apache.parquet.crypto.DecryptionPropertiesFactory +import org.apache.parquet.crypto.keytools.KeyToolkit import org.apache.spark.sql.internal.SQLConf object CometParquetUtils { private val PARQUET_FIELD_ID_WRITE_ENABLED = "spark.sql.parquet.fieldId.write.enabled" private val PARQUET_FIELD_ID_READ_ENABLED = "spark.sql.parquet.fieldId.read.enabled" private val IGNORE_MISSING_PARQUET_FIELD_ID = "spark.sql.parquet.fieldId.read.ignoreMissing" + // Map of unsupported encryption configuration key-value pairs + private val UNSUPPORTED_ENCRYPTION_CONFIGS: Map[String, Set[String]] = Map( + "parquet.encryption.algorithm" -> Set("AES_GCM_CTR_V1") Review Comment: Where do we get this from? ########## 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); Review Comment: Is this hadoop conf the entire hadoop configuration (which can have a thousand entries) or just the incremental properties specified for the session? Hashing this can become time comsuming. ########## spark/src/main/scala/org/apache/spark/sql/comet/operators.scala: ########## @@ -201,6 +207,30 @@ abstract class CometNativeExec extends CometExec { // TODO: support native metrics for all operators. val nativeMetrics = CometMetricNode.fromCometPlan(this) + // For each relation in a CometNativeScan generate a hadoopConf, + // for each file path in a relation associate with hadoopConf + val cometNativeScans: Seq[CometNativeScanExec] = this + .collectLeaves() + .filter(_.isInstanceOf[CometNativeScanExec]) + .map(_.asInstanceOf[CometNativeScanExec]) + val encryptedFilePaths = cometNativeScans.flatMap { scan => + // This creates a hadoopConf that brings in any SQLConf "spark.hadoop.*" configs and + // per-relation configs since different tables might have different decryption + // properties. + val hadoopConf = scan.relation.sparkSession.sessionState + .newHadoopConfWithOptions(scan.relation.options) + val encryptionEnabled = CometParquetUtils.encryptionEnabled(hadoopConf) + if (encryptionEnabled) { + // hadoopConf isn't serializable, so we have to do a broadcasted config. + val broadcastedConf = + scan.relation.sparkSession.sparkContext Review Comment: Can you explain a little what you are doing here? What is the additional hadoop conf information that needs to be broadcast per file path (as opposed to encryption properties that are defined once per table)? -- 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]
