Skip to content

Commit 51a205c

Browse files
committed
update
1 parent 0fea3e1 commit 51a205c

File tree

6 files changed

+347
-6
lines changed

6 files changed

+347
-6
lines changed

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

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,10 +33,8 @@
3333
import org.apache.parquet.hadoop.ParquetFileReader;
3434
import org.apache.parquet.hadoop.metadata.BlockMetaData;
3535
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
36-
import org.apache.parquet.hadoop.util.HadoopInputFile;
3736
import org.apache.parquet.internal.column.columnindex.ColumnIndex;
3837
import org.apache.parquet.internal.column.columnindex.OffsetIndex;
39-
import org.apache.parquet.io.InputFile;
4038
import org.slf4j.Logger;
4139

4240
/**
@@ -83,7 +81,7 @@ public int run() throws IOException {
8381
Preconditions.checkArgument(files != null && files.size() >= 1, "A Parquet file is required.");
8482
Preconditions.checkArgument(files.size() == 1, "Cannot process multiple Parquet files.");
8583

86-
InputFile in = HadoopInputFile.fromPath(qualifiedPath(files.get(0)), getConf());
84+
String source = files.get(0);
8785
if (!showColumnIndex && !showOffsetIndex) {
8886
showColumnIndex = true;
8987
showOffsetIndex = true;
@@ -94,7 +92,7 @@ public int run() throws IOException {
9492
rowGroupIndexSet.addAll(rowGroupIndexes);
9593
}
9694

97-
try (ParquetFileReader reader = ParquetFileReader.open(in)) {
95+
try (ParquetFileReader reader = createParquetFileReader(source)) {
9896
boolean firstBlock = true;
9997
int rowGroupIndex = 0;
10098
for (BlockMetaData block : reader.getFooter().getBlocks()) {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,7 @@ public int run() throws IOException {
6262

6363
String source = targets.get(0);
6464

65-
try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) {
65+
try (ParquetFileReader reader = createParquetFileReader(source)) {
6666
MessageType schema = reader.getFileMetaData().getSchema();
6767
ColumnDescriptor descriptor = Util.descriptor(column, schema);
6868
PrimitiveType type = Util.primitive(column, schema);

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ public int run() throws IOException {
7777
Preconditions.checkArgument(targets.size() == 1, "Cannot process multiple Parquet files.");
7878

7979
String source = targets.get(0);
80-
try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) {
80+
try (ParquetFileReader reader = createParquetFileReader(source)) {
8181
ParquetMetadata footer = reader.getFooter();
8282
MessageType schema = footer.getFileMetaData().getSchema();
8383

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

Lines changed: 114 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,10 +18,31 @@
1818
*/
1919
package org.apache.parquet.cli.commands;
2020

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

@@ -34,4 +55,97 @@ public void testShowColumnIndexCommand() throws IOException {
3455
command.setConf(new Configuration());
3556
Assert.assertEquals(0, command.run());
3657
}
58+
59+
@Test
60+
public void testEncryptedFileWithColumnIndex() throws IOException {
61+
File encryptedFile = createEncryptedFile();
62+
63+
ShowColumnIndexCommand command = new ShowColumnIndexCommand(createLogger());
64+
command.files = Arrays.asList(encryptedFile.getAbsolutePath());
65+
66+
Configuration conf = new Configuration();
67+
conf.set("parquet.encryption.footer.key", "0102030405060708090a0b0c0d0e0f10");
68+
conf.set("parquet.encryption.column.keys", "02030405060708090a0b0c0d0e0f1011:name,email");
69+
command.setConf(conf);
70+
71+
Assert.assertEquals(0, command.run());
72+
73+
encryptedFile.delete();
74+
}
75+
76+
private File createEncryptedFile() throws IOException {
77+
MessageType schema = Types.buildMessage()
78+
.required(INT32)
79+
.named("id")
80+
.required(BINARY)
81+
.named("name")
82+
.required(BINARY)
83+
.named("email")
84+
.named("test_schema");
85+
86+
File tempFile = new File(getTempFolder(), "encrypted_column_index_test.parquet");
87+
tempFile.deleteOnExit();
88+
89+
Configuration conf = new Configuration();
90+
GroupWriteSupport.setSchema(schema, conf);
91+
92+
String[] encryptColumns = {"name", "email"};
93+
FileEncryptionProperties encryptionProperties =
94+
createFileEncryptionProperties(encryptColumns, ParquetCipher.AES_GCM_CTR_V1, true);
95+
96+
SimpleGroupFactory factory = new SimpleGroupFactory(schema);
97+
98+
try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(new Path(tempFile.toURI()))
99+
.withConf(conf)
100+
.withCompressionCodec(CompressionCodecName.UNCOMPRESSED)
101+
.withEncryption(encryptionProperties)
102+
.withPageSize(1024)
103+
.withRowGroupSize(4096)
104+
.build()) {
105+
106+
for (int i = 0; i < 10; i++) {
107+
SimpleGroup group = (SimpleGroup) factory.newGroup();
108+
group.add("id", i + 1);
109+
group.add("name", Binary.fromString("name_" + i));
110+
group.add("email", Binary.fromString("email_" + i + "@test.com"));
111+
writer.write(group);
112+
}
113+
}
114+
115+
return tempFile;
116+
}
117+
118+
private FileEncryptionProperties createFileEncryptionProperties(
119+
String[] encryptColumns, ParquetCipher cipher, boolean footerEncryption) {
120+
121+
byte[] footerKey = {
122+
0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10
123+
};
124+
125+
byte[] sharedKey = new byte[] {
126+
0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11
127+
};
128+
129+
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new HashMap<>();
130+
for (String columnPath : encryptColumns) {
131+
ColumnPath column = ColumnPath.fromDotString(columnPath);
132+
133+
ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder(column)
134+
.withKey(sharedKey)
135+
.withKeyMetaData(columnPath.getBytes(StandardCharsets.UTF_8))
136+
.build();
137+
columnPropertyMap.put(column, columnProps);
138+
}
139+
140+
FileEncryptionProperties.Builder builder = FileEncryptionProperties.builder(footerKey)
141+
.withFooterKeyMetadata("footkey".getBytes(StandardCharsets.UTF_8))
142+
.withAlgorithm(cipher)
143+
.withEncryptedColumns(columnPropertyMap);
144+
145+
if (!footerEncryption) {
146+
builder.withPlaintextFooter();
147+
}
148+
149+
return builder.build();
150+
}
37151
}

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

Lines changed: 115 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,10 +18,31 @@
1818
*/
1919
package org.apache.parquet.cli.commands;
2020

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

@@ -56,4 +77,98 @@ public void testShowDirectoryCommandForFixedLengthByteArray() throws IOException
5677
command.setConf(new Configuration());
5778
Assert.assertEquals(0, command.run());
5879
}
80+
81+
@Test
82+
public void testEncryptedFileWithDictionary() throws IOException {
83+
File encryptedFile = createEncryptedFile();
84+
85+
ShowDictionaryCommand command = new ShowDictionaryCommand(createLogger());
86+
command.targets = Arrays.asList(encryptedFile.getAbsolutePath());
87+
command.column = "name";
88+
89+
Configuration conf = new Configuration();
90+
conf.set("parquet.encryption.footer.key", "0102030405060708090a0b0c0d0e0f10");
91+
conf.set("parquet.encryption.column.keys", "02030405060708090a0b0c0d0e0f1011:name,email");
92+
command.setConf(conf);
93+
94+
Assert.assertEquals(0, command.run());
95+
96+
encryptedFile.delete();
97+
}
98+
99+
private File createEncryptedFile() throws IOException {
100+
MessageType schema = Types.buildMessage()
101+
.required(INT32)
102+
.named("id")
103+
.required(BINARY)
104+
.named("name")
105+
.required(BINARY)
106+
.named("email")
107+
.named("test_schema");
108+
109+
File tempFile = new File(getTempFolder(), "encrypted_dictionary_test.parquet");
110+
tempFile.deleteOnExit();
111+
112+
Configuration conf = new Configuration();
113+
GroupWriteSupport.setSchema(schema, conf);
114+
115+
String[] encryptColumns = {"name", "email"};
116+
FileEncryptionProperties encryptionProperties =
117+
createFileEncryptionProperties(encryptColumns, ParquetCipher.AES_GCM_CTR_V1, true);
118+
119+
SimpleGroupFactory factory = new SimpleGroupFactory(schema);
120+
121+
try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(new Path(tempFile.toURI()))
122+
.withConf(conf)
123+
.withCompressionCodec(CompressionCodecName.UNCOMPRESSED)
124+
.withEncryption(encryptionProperties)
125+
.withPageSize(1024)
126+
.withRowGroupSize(4096)
127+
.build()) {
128+
129+
for (int i = 0; i < 10; i++) {
130+
SimpleGroup group = (SimpleGroup) factory.newGroup();
131+
group.add("id", i + 1);
132+
group.add("name", Binary.fromString("name_" + (i % 3)));
133+
group.add("email", Binary.fromString("email_" + (i % 3) + "@test.com"));
134+
writer.write(group);
135+
}
136+
}
137+
138+
return tempFile;
139+
}
140+
141+
private FileEncryptionProperties createFileEncryptionProperties(
142+
String[] encryptColumns, ParquetCipher cipher, boolean footerEncryption) {
143+
144+
byte[] footerKey = {
145+
0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10
146+
};
147+
148+
byte[] sharedKey = new byte[] {
149+
0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11
150+
};
151+
152+
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new HashMap<>();
153+
for (String columnPath : encryptColumns) {
154+
ColumnPath column = ColumnPath.fromDotString(columnPath);
155+
156+
ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder(column)
157+
.withKey(sharedKey)
158+
.withKeyMetaData(columnPath.getBytes(StandardCharsets.UTF_8))
159+
.build();
160+
columnPropertyMap.put(column, columnProps);
161+
}
162+
163+
FileEncryptionProperties.Builder builder = FileEncryptionProperties.builder(footerKey)
164+
.withFooterKeyMetadata("footkey".getBytes(StandardCharsets.UTF_8))
165+
.withAlgorithm(cipher)
166+
.withEncryptedColumns(columnPropertyMap);
167+
168+
if (!footerEncryption) {
169+
builder.withPlaintextFooter();
170+
}
171+
172+
return builder.build();
173+
}
59174
}

0 commit comments

Comments
 (0)