Skip to content

Commit 3f46fc5

Browse files
committed
update
1 parent 4147bc6 commit 3f46fc5

16 files changed

+283
-23
lines changed

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

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,6 @@
4646
import org.apache.parquet.column.page.PageReadStore;
4747
import org.apache.parquet.column.page.PageReader;
4848
import org.apache.parquet.column.statistics.Statistics;
49-
import org.apache.parquet.format.converter.ParquetMetadataConverter;
5049
import org.apache.parquet.hadoop.ParquetFileReader;
5150
import org.apache.parquet.hadoop.metadata.FileMetaData;
5251
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
@@ -86,7 +85,10 @@ public int run() throws IOException {
8685

8786
private String check(String file) throws IOException {
8887
Path path = qualifiedPath(file);
89-
ParquetMetadata footer = ParquetFileReader.readFooter(getConf(), path, ParquetMetadataConverter.NO_FILTER);
88+
ParquetMetadata footer;
89+
try (ParquetFileReader reader = createParquetFileReader(file)) {
90+
footer = reader.getFooter();
91+
}
9092

9193
FileMetaData meta = footer.getFileMetaData();
9294
String createdBy = meta.getCreatedBy();

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

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -27,10 +27,8 @@
2727
import java.util.HashMap;
2828
import java.util.List;
2929
import java.util.Map;
30-
import org.apache.hadoop.conf.Configuration;
3130
import org.apache.hadoop.fs.Path;
3231
import org.apache.parquet.cli.BaseCommand;
33-
import org.apache.parquet.format.converter.ParquetMetadataConverter;
3432
import org.apache.parquet.hadoop.ParquetFileReader;
3533
import org.apache.parquet.hadoop.metadata.BlockMetaData;
3634
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
@@ -102,13 +100,14 @@ public List<String> getExamples() {
102100
// Make it public to allow some automation tools to call it
103101
public Map<String, Long> getColumnSizeInBytes(Path inputFile) throws IOException {
104102
Map<String, Long> colSizes = new HashMap<>();
105-
ParquetMetadata pmd =
106-
ParquetFileReader.readFooter(new Configuration(), inputFile, ParquetMetadataConverter.NO_FILTER);
107103

108-
for (BlockMetaData block : pmd.getBlocks()) {
109-
for (ColumnChunkMetaData column : block.getColumns()) {
110-
String colName = column.getPath().toDotString();
111-
colSizes.put(colName, column.getTotalSize() + colSizes.getOrDefault(colName, 0L));
104+
try (ParquetFileReader reader = createParquetFileReader(inputFile.toString())) {
105+
ParquetMetadata pmd = reader.getFooter();
106+
for (BlockMetaData block : pmd.getBlocks()) {
107+
for (ColumnChunkMetaData column : block.getColumns()) {
108+
String colName = column.getPath().toDotString();
109+
colSizes.put(colName, column.getTotalSize() + colSizes.getOrDefault(colName, 0L));
110+
}
112111
}
113112
}
114113

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

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,6 @@
3131
import org.apache.avro.file.SeekableInput;
3232
import org.apache.parquet.cli.BaseCommand;
3333
import org.apache.parquet.cli.util.Formats;
34-
import org.apache.parquet.format.converter.ParquetMetadataConverter;
3534
import org.apache.parquet.hadoop.ParquetFileReader;
3635
import org.slf4j.Logger;
3736

@@ -113,8 +112,7 @@ private String getParquetSchema(String source) throws IOException {
113112

114113
switch (format) {
115114
case PARQUET:
116-
try (ParquetFileReader reader = new ParquetFileReader(
117-
getConf(), qualifiedPath(source), ParquetMetadataConverter.NO_FILTER)) {
115+
try (ParquetFileReader reader = createParquetFileReader(source)) {
118116
return reader.getFileMetaData().getSchema().toString();
119117
}
120118
default:

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

Lines changed: 1 addition & 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,6 @@ 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());
8784
if (!showColumnIndex && !showOffsetIndex) {
8885
showColumnIndex = true;
8986
showOffsetIndex = true;
@@ -94,7 +91,7 @@ public int run() throws IOException {
9491
rowGroupIndexSet.addAll(rowGroupIndexes);
9592
}
9693

97-
try (ParquetFileReader reader = ParquetFileReader.open(in)) {
94+
try (ParquetFileReader reader = createParquetFileReader(files.get(0))) {
9895
boolean firstBlock = true;
9996
int rowGroupIndex = 0;
10097
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/ShowFooterCommand.java

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

6767
private String readFooter(InputFile inputFile) throws JsonProcessingException, IOException {
6868
String json;
69-
try (ParquetFileReader reader = ParquetFileReader.open(inputFile)) {
69+
try (ParquetFileReader reader = createParquetFileReader(target)) {
7070
ParquetMetadata footer = reader.getFooter();
7171
ObjectMapper mapper = RawUtils.createObjectMapper();
7272
mapper.setVisibility(PropertyAccessor.ALL, Visibility.NONE);

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

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

5353
String source = targets.get(0);
54-
try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) {
54+
try (ParquetFileReader reader = createParquetFileReader(source)) {
5555
ParquetMetadata footer = reader.getFooter();
5656
MessageType schema = footer.getFileMetaData().getSchema();
5757

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/ColumnSizeCommandTest.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,7 @@ public void testColumnSizeCommand() throws IOException {
5757
@Test
5858
public void testColumnSize() throws Exception {
5959
String inputFile = createParquetFile();
60+
command.setConf(conf);
6061
Map<String, Long> columnSizeInBytes = command.getColumnSizeInBytes(new Path(inputFile));
6162
assertEquals(columnSizeInBytes.size(), 2);
6263
assertTrue(columnSizeInBytes.get("DocId") > columnSizeInBytes.get("Num"));
@@ -85,4 +86,18 @@ private String createParquetFile() throws IOException {
8586

8687
return file;
8788
}
89+
90+
@Test
91+
public void testColumnSizeCommandWithEncryptedFile() throws IOException {
92+
File encryptedFile = EncryptedParquetFileTestHelper.createEncryptedParquetFile(
93+
getTempFolder(), "encrypted_columnsize_test.parquet");
94+
95+
ColumnSizeCommand command = new ColumnSizeCommand(createLogger());
96+
command.target = encryptedFile.getAbsolutePath();
97+
command.setConf(EncryptedParquetFileTestHelper.createDecryptionConfiguration());
98+
99+
Assert.assertEquals(0, command.run());
100+
101+
encryptedFile.delete();
102+
}
88103
}
Lines changed: 177 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,177 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.parquet.cli.commands;
20+
21+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
22+
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
23+
24+
import java.io.File;
25+
import java.io.IOException;
26+
import java.nio.charset.StandardCharsets;
27+
import java.util.HashMap;
28+
import java.util.Map;
29+
import org.apache.hadoop.conf.Configuration;
30+
import org.apache.hadoop.fs.Path;
31+
import org.apache.parquet.crypto.ColumnEncryptionProperties;
32+
import org.apache.parquet.crypto.FileEncryptionProperties;
33+
import org.apache.parquet.crypto.ParquetCipher;
34+
import org.apache.parquet.example.data.Group;
35+
import org.apache.parquet.example.data.simple.SimpleGroup;
36+
import org.apache.parquet.example.data.simple.SimpleGroupFactory;
37+
import org.apache.parquet.hadoop.ParquetWriter;
38+
import org.apache.parquet.hadoop.example.ExampleParquetWriter;
39+
import org.apache.parquet.hadoop.example.GroupWriteSupport;
40+
import org.apache.parquet.hadoop.metadata.ColumnPath;
41+
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
42+
import org.apache.parquet.io.api.Binary;
43+
import org.apache.parquet.schema.MessageType;
44+
import org.apache.parquet.schema.Types;
45+
46+
/**
47+
* Utility class for creating encrypted Parquet files for testing CLI commands.
48+
*/
49+
public final class EncryptedParquetFileTestHelper {
50+
51+
// Standard test encryption keys (16 bytes for AES-128)
52+
public static final byte[] FOOTER_KEY = {
53+
0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10
54+
};
55+
56+
public static final byte[] COLUMN_KEY_1 = {
57+
0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11
58+
};
59+
60+
public static final byte[] COLUMN_KEY_2 = {
61+
0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11, 0x12, 0x13
62+
};
63+
64+
public static final String FOOTER_KEY_HEX = "0102030405060708090a0b0c0d0e0f10";
65+
public static final String COLUMN_KEY_1_HEX = "02030405060708090a0b0c0d0e0f1011";
66+
public static final String COLUMN_KEY_2_HEX = "0405060708090a0b0c0d0e0f10111213";
67+
68+
public static final String COLUMN_KEYS_CONFIG = COLUMN_KEY_1_HEX + ":name,email;" + COLUMN_KEY_2_HEX + ":phone";
69+
70+
private EncryptedParquetFileTestHelper() {}
71+
72+
public static File createEncryptedParquetFile(File tempDir, String filename) throws IOException {
73+
return createEncryptedParquetFile(tempDir, filename, true, true);
74+
}
75+
76+
public static File createEncryptedParquetFile(
77+
File tempDir, String filename, boolean enableBloomFilter, boolean encryptedFooter) throws IOException {
78+
79+
MessageType schema = Types.buildMessage()
80+
.required(INT32)
81+
.named("id")
82+
.required(BINARY)
83+
.named("name")
84+
.required(BINARY)
85+
.named("email")
86+
.required(BINARY)
87+
.named("phone")
88+
.named("test_schema");
89+
90+
File file = new File(tempDir, filename);
91+
file.deleteOnExit();
92+
93+
Configuration conf = new Configuration();
94+
GroupWriteSupport.setSchema(schema, conf);
95+
96+
String[] encryptColumns = {"name", "email", "phone"};
97+
FileEncryptionProperties encryptionProperties =
98+
createFileEncryptionProperties(encryptColumns, ParquetCipher.AES_GCM_CTR_V1, encryptedFooter);
99+
100+
SimpleGroupFactory factory = new SimpleGroupFactory(schema);
101+
String[] nameValues = {"test_value_1", "test_value_2", "another_test", "bloom_filter_test", "final_value"};
102+
String[] emailValues = {
103+
104+
};
105+
String[] phoneValues = {"555-0001", "555-0002", "555-0003", "555-0004", "555-0005"};
106+
107+
ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(new Path(file.toURI()))
108+
.withConf(conf)
109+
.withCompressionCodec(CompressionCodecName.UNCOMPRESSED)
110+
.withEncryption(encryptionProperties)
111+
.withPageSize(1024)
112+
.withRowGroupSize(4096);
113+
114+
if (enableBloomFilter) {
115+
builder.withBloomFilterEnabled("name", true)
116+
.withBloomFilterEnabled("email", true)
117+
.withBloomFilterEnabled("phone", true);
118+
}
119+
120+
try (ParquetWriter<Group> writer = builder.build()) {
121+
for (int i = 0; i < nameValues.length; i++) {
122+
SimpleGroup group = (SimpleGroup) factory.newGroup();
123+
group.add("id", i + 1);
124+
group.add("name", Binary.fromString(nameValues[i]));
125+
group.add("email", Binary.fromString(emailValues[i]));
126+
group.add("phone", Binary.fromString(phoneValues[i]));
127+
writer.write(group);
128+
}
129+
}
130+
131+
return file;
132+
}
133+
134+
public static FileEncryptionProperties createFileEncryptionProperties(
135+
String[] encryptColumns, ParquetCipher cipher, boolean footerEncryption) {
136+
137+
Map<String, byte[]> columnKeys = new HashMap<>();
138+
columnKeys.put("name", COLUMN_KEY_1);
139+
columnKeys.put("email", COLUMN_KEY_1);
140+
columnKeys.put("phone", COLUMN_KEY_2);
141+
142+
Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new HashMap<>();
143+
for (String columnPath : encryptColumns) {
144+
ColumnPath column = ColumnPath.fromDotString(columnPath);
145+
byte[] columnKey = columnKeys.get(columnPath);
146+
147+
ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder(column)
148+
.withKey(columnKey)
149+
.withKeyMetaData(columnPath.getBytes(StandardCharsets.UTF_8))
150+
.build();
151+
columnPropertyMap.put(column, columnProps);
152+
}
153+
154+
FileEncryptionProperties.Builder builder = FileEncryptionProperties.builder(FOOTER_KEY)
155+
.withFooterKeyMetadata("footkey".getBytes(StandardCharsets.UTF_8))
156+
.withAlgorithm(cipher)
157+
.withEncryptedColumns(columnPropertyMap);
158+
159+
if (!footerEncryption) {
160+
builder.withPlaintextFooter();
161+
}
162+
163+
return builder.build();
164+
}
165+
166+
public static Configuration createDecryptionConfiguration() {
167+
Configuration conf = new Configuration();
168+
conf.set("parquet.encryption.footer.key", FOOTER_KEY_HEX);
169+
conf.set("parquet.encryption.column.keys", COLUMN_KEYS_CONFIG);
170+
return conf;
171+
}
172+
173+
public static void setDecryptionProperties(Configuration conf) {
174+
conf.set("parquet.encryption.footer.key", FOOTER_KEY_HEX);
175+
conf.set("parquet.encryption.column.keys", COLUMN_KEYS_CONFIG);
176+
}
177+
}

0 commit comments

Comments
 (0)