forked from apache/datafusion-comet
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathCometFileKeyUnwrapper.java
More file actions
146 lines (136 loc) · 9.47 KB
/
CometFileKeyUnwrapper.java
File metadata and controls
146 lines (136 loc) · 9.47 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
/*
* 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<>();
// Cache the factory since we should be using the same hadoopConf for every file in this scan.
private DecryptionPropertiesFactory factory = null;
// Cache the hadoopConf just to assert the assumption above.
private Configuration conf = null;
/**
* 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
if (factory == null) {
factory = DecryptionPropertiesFactory.loadFactory(hadoopConf);
conf = hadoopConf;
} else {
// Check the assumption that all files have the same hadoopConf and thus same Factory
assert (conf == hadoopConf);
}
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);
}
}