Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
dcc882b
Parquet Modular Encryption support for native readers using KMS on Sp…
mbutrovich Sep 23, 2025
8bac76a
Fix unused import.
mbutrovich Sep 23, 2025
40935df
Fix encryptionEnabled check in NativeBatchReader.java, and guard encr…
mbutrovich Sep 23, 2025
7cbfb1b
Fix NPE when checking encryptedEnabled.
mbutrovich Sep 23, 2025
1e1fa2f
Merge branch 'main' into decryption
mbutrovich Sep 24, 2025
090497b
Minor refactor for encryptionEnabled.
mbutrovich Sep 24, 2025
992a4e1
Merge branch 'main' into decryption
mbutrovich Sep 26, 2025
c9dfdd5
More tests.
mbutrovich Sep 26, 2025
bf0bec4
Cleanup Seq loop that wasn't doing anything.
mbutrovich Sep 26, 2025
a0e2d9a
Merge branch 'main' into decryption
mbutrovich Sep 26, 2025
271e940
Docs.
mbutrovich Sep 26, 2025
571c881
Docs.
mbutrovich Sep 26, 2025
4dde7fb
Refactor out of parquet_exec.rs.
mbutrovich Sep 26, 2025
ac566f5
Merge branch 'main' into decryption
mbutrovich Sep 29, 2025
9bc24fd
Add uniform encryption test.
mbutrovich Sep 29, 2025
1dfb252
Merge branch 'main' into decryption
mbutrovich Sep 30, 2025
bf6ad03
Address PR feedback.
mbutrovich Sep 30, 2025
7d1bf39
Add benchmark.
mbutrovich Sep 30, 2025
8ba2680
Address PR feedback related to number of hadoopConfs in a Comet plan …
mbutrovich Oct 1, 2025
e9fcca7
Adjust error handling.
mbutrovich Oct 1, 2025
e8c23ab
Add test with UNION.
mbutrovich Oct 6, 2025
a53aa2f
Add docs to reflect UNION discussion in PR feedback.
mbutrovich Oct 6, 2025
fbe2c96
Address PR feedback.
mbutrovich Oct 6, 2025
d266184
Merge branch 'main' into decryption
mbutrovich Oct 6, 2025
257f163
Fix after merge conflicts with main.
mbutrovich Oct 6, 2025
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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;

/**
* Helper class to access DecryptionKeyRetriever.getKey from native code via JNI. This class handles
* the complexity of getting the proper Hadoop Configuration from the current Spark context and
* creating properly configured DecryptionKeyRetriever instances using DecryptionPropertiesFactory.
*/
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 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);
}
Path path = new Path(filePath);
FileDecryptionProperties decryptionProperties =
factory.getFileDecryptionProperties(hadoopConf, path);

DecryptionKeyRetriever keyRetriever = decryptionProperties.getKeyRetriever();
retrieverCache.put(filePath, keyRetriever);
}

/**
* Gets the decryption key for the given key metadata using the cached DecryptionKeyRetriever for
* the specified file path.
*
* @param filePath The path to the Parquet file
* @param keyMetadata The key metadata bytes from the Parquet file
* @return The decrypted key bytes
* @throws ParquetCryptoRuntimeException if key unwrapping fails
*/
public byte[] getKey(final String filePath, final byte[] keyMetadata)
throws ParquetCryptoRuntimeException {
DecryptionKeyRetriever keyRetriever = retrieverCache.get(filePath);
if (keyRetriever == null) {
throw new ParquetCryptoRuntimeException(
"Failed to find DecryptionKeyRetriever for path: " + filePath);
}
return keyRetriever.getKey(keyMetadata);
}
}
5 changes: 4 additions & 1 deletion common/src/main/java/org/apache/comet/parquet/Native.java
Original file line number Diff line number Diff line change
Expand Up @@ -267,9 +267,11 @@ public static native long initRecordBatchReader(
String sessionTimezone,
int batchSize,
boolean caseSensitive,
Map<String, String> objectStoreOptions);
Map<String, String> objectStoreOptions,
CometFileKeyUnwrapper keyUnwrapper);

// arrow native version of read batch

/**
* Read the next batch of data into memory on native side
*
Expand All @@ -280,6 +282,7 @@ public static native long initRecordBatchReader(

// arrow native equivalent of currentBatch. 'columnNum' is number of the column in the record
// batch

/**
* Load the column corresponding to columnNum in the currently loaded record batch into JVM
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@
import org.apache.comet.vector.CometVector;
import org.apache.comet.vector.NativeUtil;

import static scala.jdk.javaapi.CollectionConverters.*;
import static scala.jdk.javaapi.CollectionConverters.asJava;

/**
* A vectorized Parquet reader that reads a Parquet file in a batched fashion.
Expand Down Expand Up @@ -410,6 +410,16 @@ public void init() throws Throwable {
}
}

boolean encryptionEnabled =
!conf.get("parquet.crypto.factory.class").isEmpty()
|| !conf.get("parquet.encryption.kms.client.class", "").isEmpty();

// Create keyUnwrapper if encryption is enabled
CometFileKeyUnwrapper keyUnwrapper = encryptionEnabled ? new CometFileKeyUnwrapper() : null;
if (encryptionEnabled) {
keyUnwrapper.storeDecryptionKeyRetriever(file.filePath().toString(), conf);
}

int batchSize =
conf.getInt(
CometConf.COMET_BATCH_SIZE().key(),
Expand All @@ -426,7 +436,8 @@ public void init() throws Throwable {
timeZoneId,
batchSize,
caseSensitive,
objectStoreOptions);
objectStoreOptions,
keyUnwrapper);
}
isInitialized = true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ object NativeConfig {
def extractObjectStoreOptions(hadoopConf: Configuration, uri: URI): Map[String, String] = {
val scheme = uri.getScheme.toLowerCase(Locale.ROOT)

import scala.collection.JavaConverters._
import scala.jdk.CollectionConverters._
val options = scala.collection.mutable.Map[String, String]()

// The schemes will use libhdfs
Expand Down
4 changes: 4 additions & 0 deletions native/Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion native/core/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ bytes = { workspace = true }
tempfile = "3.8.0"
itertools = "0.14.0"
paste = "1.0.14"
datafusion = { workspace = true }
datafusion = { workspace = true, features = ["parquet_encryption"] }
datafusion-spark = { workspace = true }
once_cell = "1.18.0"
regex = { workspace = true }
Expand Down
13 changes: 13 additions & 0 deletions native/core/src/execution/jni_api.rs
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ use crate::execution::spark_plan::SparkPlan;

use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace};

use crate::parquet::parquet_exec::{CometEncryptionFactory, ENCRYPTION_FACTORY_ID};
use datafusion_comet_proto::spark_operator::operator::OpStruct;
use log::info;
use once_cell::sync::Lazy;
Expand Down Expand Up @@ -167,6 +168,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
debug_native: jboolean,
explain_native: jboolean,
tracing_enabled: jboolean,
key_unwrapper_obj: JObject,
) -> jlong {
try_unwrap_or_throw(&e, |mut env| {
with_trace("createPlan", tracing_enabled != JNI_FALSE, || {
Expand Down Expand Up @@ -239,6 +241,17 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
None
};

// Handle key unwrapper for encrypted files
if !key_unwrapper_obj.is_null() {
let encryption_factory = CometEncryptionFactory {
key_unwrapper: jni_new_global_ref!(env, key_unwrapper_obj)?,
};
session.runtime_env().register_parquet_encryption_factory(
ENCRYPTION_FACTORY_ID,
Arc::new(encryption_factory),
);
}

let exec_context = Box::new(ExecutionContext {
id,
task_attempt_id,
Expand Down
2 changes: 2 additions & 0 deletions native/core/src/execution/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1360,6 +1360,8 @@ impl PhysicalPlanner {
default_values,
scan.session_timezone.as_str(),
scan.case_sensitive,
self.session_ctx(),
scan.encryption_enabled,
)?;
Ok((
vec![],
Expand Down
25 changes: 24 additions & 1 deletion native/core/src/parquet/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,11 @@ use crate::execution::operators::ExecutionError;
use crate::execution::planner::PhysicalPlanner;
use crate::execution::serde;
use crate::execution::utils::SparkArrowConvert;
use crate::jvm_bridge::{jni_new_global_ref, JVMClasses};
use crate::parquet::data_type::AsBytes;
use crate::parquet::parquet_exec::init_datasource_exec;
use crate::parquet::parquet_exec::{
init_datasource_exec, CometEncryptionFactory, ENCRYPTION_FACTORY_ID,
};
use crate::parquet::parquet_support::prepare_object_store_with_configs;
use arrow::array::{Array, RecordBatch};
use arrow::buffer::{Buffer, MutableBuffer};
Expand Down Expand Up @@ -712,8 +715,10 @@ pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_initRecordBat
batch_size: jint,
case_sensitive: jboolean,
object_store_options: jobject,
key_unwrapper_obj: JObject,
) -> jlong {
try_unwrap_or_throw(&e, |mut env| unsafe {
JVMClasses::init(&mut env);
let session_config = SessionConfig::new().with_batch_size(batch_size as usize);
let planner =
PhysicalPlanner::new(Arc::new(SessionContext::new_with_config(session_config)), 0);
Expand Down Expand Up @@ -766,6 +771,22 @@ pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_initRecordBat
.unwrap()
.into();

// Handle key unwrapper for encrypted files
let encryption_enabled = if !key_unwrapper_obj.is_null() {
let encryption_factory = CometEncryptionFactory {
key_unwrapper: jni_new_global_ref!(env, key_unwrapper_obj)?,
};
session_ctx
.runtime_env()
.register_parquet_encryption_factory(
ENCRYPTION_FACTORY_ID,
Arc::new(encryption_factory),
);
true
} else {
false
};

let scan = init_datasource_exec(
required_schema,
Some(data_schema),
Expand All @@ -778,6 +799,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_initRecordBat
None,
session_timezone.as_str(),
case_sensitive != JNI_FALSE,
session_ctx,
encryption_enabled,
)?;

let partition_index: usize = 0;
Expand Down
Loading
Loading