Skip to content

Commit cf06df8

Browse files
committed
update
1 parent d5df847 commit cf06df8

File tree

2 files changed

+186
-2
lines changed

2 files changed

+186
-2
lines changed

parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowBloomFilterCommand.java

Lines changed: 71 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,12 +24,16 @@
2424
import com.google.common.base.Preconditions;
2525
import com.google.common.collect.Lists;
2626
import java.io.IOException;
27+
import java.nio.charset.StandardCharsets;
2728
import java.util.List;
2829
import java.util.Optional;
2930
import org.apache.commons.text.TextStringBuilder;
31+
import org.apache.parquet.HadoopReadOptions;
3032
import org.apache.parquet.cli.BaseCommand;
3133
import org.apache.parquet.cli.Util;
3234
import org.apache.parquet.column.values.bloomfilter.BloomFilter;
35+
import org.apache.parquet.crypto.DecryptionKeyRetriever;
36+
import org.apache.parquet.crypto.FileDecryptionProperties;
3337
import org.apache.parquet.hadoop.BloomFilterReader;
3438
import org.apache.parquet.hadoop.ParquetFileReader;
3539
import org.apache.parquet.hadoop.metadata.BlockMetaData;
@@ -63,14 +67,31 @@ public ShowBloomFilterCommand(Logger console) {
6367
required = true)
6468
List<String> testValues;
6569

70+
@Parameter(
71+
names = {"--footer-key"},
72+
description = "Footer key for encrypted files (hex string, 16/24/32 bytes)")
73+
String footerKeyHex;
74+
75+
@Parameter(
76+
names = {"--column-key"},
77+
description = "Column key for encrypted files (hex string, 16/24/32 bytes)")
78+
String columnKeyHex;
79+
6680
@Override
6781
@SuppressWarnings("unchecked")
6882
public int run() throws IOException {
6983
Preconditions.checkArgument(file != null, "A Parquet file is required.");
7084

7185
InputFile in = HadoopInputFile.fromPath(qualifiedPath(file), getConf());
7286

73-
try (ParquetFileReader reader = ParquetFileReader.open(in)) {
87+
FileDecryptionProperties decryptionProperties = createDecryptionProperties();
88+
89+
HadoopReadOptions.Builder optionsBuilder = HadoopReadOptions.builder(getConf());
90+
if (decryptionProperties != null) {
91+
optionsBuilder.withDecryption(decryptionProperties);
92+
}
93+
94+
try (ParquetFileReader reader = ParquetFileReader.open(in, optionsBuilder.build())) {
7495
MessageType schema = reader.getFileMetaData().getSchema();
7596
PrimitiveType type = Util.primitive(columnPath, schema);
7697

@@ -122,9 +143,57 @@ private Object getOriginalType(String value, PrimitiveType type) {
122143
}
123144
}
124145

146+
private FileDecryptionProperties createDecryptionProperties() {
147+
if (footerKeyHex == null && columnKeyHex == null) {
148+
return null;
149+
}
150+
151+
SimpleKeyRetriever keyRetriever = new SimpleKeyRetriever();
152+
153+
FileDecryptionProperties.Builder builder =
154+
FileDecryptionProperties.builder().withPlaintextFilesAllowed();
155+
156+
byte[] footerKey = hexToBytes(footerKeyHex);
157+
builder.withFooterKey(footerKey);
158+
159+
byte[] columnKey = hexToBytes(columnKeyHex);
160+
keyRetriever.putKey("col", columnKey);
161+
builder.withKeyRetriever(keyRetriever);
162+
163+
return builder.build();
164+
}
165+
166+
private static byte[] hexToBytes(String hex) {
167+
int len = hex.length();
168+
byte[] data = new byte[len / 2];
169+
for (int i = 0; i < len; i += 2) {
170+
data[i / 2] = (byte) ((Character.digit(hex.charAt(i), 16) << 4) | Character.digit(hex.charAt(i + 1), 16));
171+
}
172+
return data;
173+
}
174+
175+
private static class SimpleKeyRetriever implements DecryptionKeyRetriever {
176+
private final java.util.Map<String, byte[]> keyMap = new java.util.HashMap<>();
177+
178+
public void putKey(String keyId, byte[] keyBytes) {
179+
keyMap.put(keyId, keyBytes);
180+
}
181+
182+
@Override
183+
public byte[] getKey(byte[] keyMetaData) {
184+
String keyId = new String(keyMetaData, StandardCharsets.UTF_8);
185+
return keyMap.get(keyId);
186+
}
187+
}
188+
125189
@Override
126190
public List<String> getExamples() {
127191
return Lists.newArrayList(
128-
"# Show bloom filter for column 'col' from a Parquet file", "-c col -v 1,2,3 -i sample.parquet");
192+
"# Show bloom filter for column 'col' from a Parquet file",
193+
"-c col -v 1,2,3 sample.parquet",
194+
"# Show bloom filter for encrypted file with footer key",
195+
"-c col -v 1,2,3 --footer-key 0102030405060708090a0b0c0d0e0f10 encrypted.parquet",
196+
"# Show bloom filter for encrypted file with column key",
197+
"-c col -v 1,2,3 --column-key 0102030405060708090a0b0c0d0e0f10 encrypted.parquet");
129198
}
130199
}

parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowBloomFilterCommandTest.java

Lines changed: 115 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,10 +19,31 @@
1919

2020
package org.apache.parquet.cli.commands;
2121

22+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
23+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
24+
2225
import java.io.File;
2326
import java.io.IOException;
27+
import java.nio.charset.StandardCharsets;
2428
import java.util.Arrays;
29+
import java.util.HashMap;
30+
import java.util.Map;
2531
import org.apache.hadoop.conf.Configuration;
32+
import org.apache.hadoop.fs.Path;
33+
import org.apache.parquet.crypto.ColumnEncryptionProperties;
34+
import org.apache.parquet.crypto.FileEncryptionProperties;
35+
import org.apache.parquet.crypto.ParquetCipher;
36+
import org.apache.parquet.example.data.Group;
37+
import org.apache.parquet.example.data.simple.SimpleGroup;
38+
import org.apache.parquet.example.data.simple.SimpleGroupFactory;
39+
import org.apache.parquet.hadoop.ParquetWriter;
40+
import org.apache.parquet.hadoop.example.ExampleParquetWriter;
41+
import org.apache.parquet.hadoop.example.GroupWriteSupport;
42+
import org.apache.parquet.hadoop.metadata.ColumnPath;
43+
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
44+
import org.apache.parquet.io.api.Binary;
45+
import org.apache.parquet.schema.MessageType;
46+
import org.apache.parquet.schema.Types;
2647
import org.junit.Assert;
2748
import org.junit.Test;
2849

@@ -37,4 +58,98 @@ public void testShowBloomFilterCommand() throws IOException {
3758
command.setConf(new Configuration());
3859
Assert.assertEquals(0, command.run());
3960
}
61+
62+
@Test
63+
public void testEncryptedFileWithBloomFilter() throws IOException {
64+
File encryptedFile = createEncryptedFileWithBloomFilter();
65+
66+
ShowBloomFilterCommand command = new ShowBloomFilterCommand(createLogger());
67+
command.file = encryptedFile.getAbsolutePath();
68+
command.columnPath = "name";
69+
command.testValues = Arrays.asList(new String[] {"test_value_1", "non_existent_value"});
70+
71+
command.footerKeyHex = "0102030405060708090a0b0c0d0e0f10";
72+
command.columnKeyHex = "02030405060708090a0b0c0d0e0f1011";
73+
command.setConf(new Configuration());
74+
75+
Assert.assertEquals(0, command.run());
76+
77+
encryptedFile.delete();
78+
}
79+
80+
/**
81+
* Creates an encrypted Parquet file with bloom filters for testing
82+
*/
83+
private File createEncryptedFileWithBloomFilter() throws IOException {
84+
MessageType schema = Types.buildMessage()
85+
.required(INT32)
86+
.named("id")
87+
.required(BINARY)
88+
.named("name")
89+
.named("test_schema");
90+
91+
File tempFile = new File(getTempFolder(), "encrypted_bloom_test.parquet");
92+
tempFile.deleteOnExit();
93+
94+
Configuration conf = new Configuration();
95+
GroupWriteSupport.setSchema(schema, conf);
96+
97+
String[] encryptColumns = {"name"};
98+
FileEncryptionProperties encryptionProperties =
99+
createFileEncryptionProperties(encryptColumns, ParquetCipher.AES_GCM_CTR_V1, true);
100+
101+
SimpleGroupFactory factory = new SimpleGroupFactory(schema);
102+
String[] testValues = {"test_value_1", "test_value_2", "another_test", "bloom_filter_test", "final_value"};
103+
104+
try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(new Path(tempFile.toURI()))
105+
.withConf(conf)
106+
.withCompressionCodec(CompressionCodecName.UNCOMPRESSED)
107+
.withEncryption(encryptionProperties)
108+
.withBloomFilterEnabled("name", true)
109+
.withPageSize(1024)
110+
.withRowGroupSize(4096)
111+
.build()) {
112+
113+
for (int i = 0; i < testValues.length; i++) {
114+
SimpleGroup group = (SimpleGroup) factory.newGroup();
115+
group.add("id", i + 1);
116+
group.add("name", Binary.fromString(testValues[i]));
117+
writer.write(group);
118+
}
119+
}
120+
121+
return tempFile;
122+
}
123+
124+
private FileEncryptionProperties createFileEncryptionProperties(
125+
String[] encryptColumns, ParquetCipher cipher, boolean footerEncryption) {
126+
127+
byte[] footerKey = {
128+
0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10
129+
};
130+
byte[] columnKey = {
131+
0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11
132+
};
133+
134+
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new HashMap<>();
135+
for (String columnPath : encryptColumns) {
136+
ColumnPath column = ColumnPath.fromDotString(columnPath);
137+
ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder(column)
138+
.withKey(columnKey)
139+
.withKeyMetaData("col".getBytes(StandardCharsets.UTF_8))
140+
.build();
141+
columnPropertyMap.put(column, columnProps);
142+
}
143+
144+
FileEncryptionProperties.Builder builder = FileEncryptionProperties.builder(footerKey)
145+
.withFooterKeyMetadata("footkey".getBytes(StandardCharsets.UTF_8))
146+
.withAlgorithm(cipher)
147+
.withEncryptedColumns(columnPropertyMap);
148+
149+
if (!footerEncryption) {
150+
builder.withPlaintextFooter();
151+
}
152+
153+
return builder.build();
154+
}
40155
}

0 commit comments

Comments
 (0)