Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
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
Expand Up @@ -46,7 +46,6 @@
import org.apache.parquet.column.page.PageReadStore;
import org.apache.parquet.column.page.PageReader;
import org.apache.parquet.column.statistics.Statistics;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.metadata.FileMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
Expand Down Expand Up @@ -86,7 +85,10 @@ public int run() throws IOException {

private String check(String file) throws IOException {
Path path = qualifiedPath(file);
ParquetMetadata footer = ParquetFileReader.readFooter(getConf(), path, ParquetMetadataConverter.NO_FILTER);
ParquetMetadata footer;
try (ParquetFileReader reader = createParquetFileReader(file)) {
footer = reader.getFooter();
}

FileMetaData meta = footer.getFileMetaData();
String createdBy = meta.getCreatedBy();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,8 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.cli.BaseCommand;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
Expand Down Expand Up @@ -102,13 +100,14 @@ public List<String> getExamples() {
// Make it public to allow some automation tools to call it
public Map<String, Long> getColumnSizeInBytes(Path inputFile) throws IOException {
Map<String, Long> colSizes = new HashMap<>();
ParquetMetadata pmd =
ParquetFileReader.readFooter(new Configuration(), inputFile, ParquetMetadataConverter.NO_FILTER);

for (BlockMetaData block : pmd.getBlocks()) {
for (ColumnChunkMetaData column : block.getColumns()) {
String colName = column.getPath().toDotString();
colSizes.put(colName, column.getTotalSize() + colSizes.getOrDefault(colName, 0L));
try (ParquetFileReader reader = createParquetFileReader(inputFile.toString())) {
ParquetMetadata pmd = reader.getFooter();
for (BlockMetaData block : pmd.getBlocks()) {
for (ColumnChunkMetaData column : block.getColumns()) {
String colName = column.getPath().toDotString();
colSizes.put(colName, column.getTotalSize() + colSizes.getOrDefault(colName, 0L));
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import org.apache.avro.file.SeekableInput;
import org.apache.parquet.cli.BaseCommand;
import org.apache.parquet.cli.util.Formats;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.slf4j.Logger;

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

switch (format) {
case PARQUET:
try (ParquetFileReader reader = new ParquetFileReader(
getConf(), qualifiedPath(source), ParquetMetadataConverter.NO_FILTER)) {
try (ParquetFileReader reader = createParquetFileReader(source)) {
return reader.getFileMetaData().getSchema().toString();
}
default:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,10 +33,8 @@
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.util.HadoopInputFile;
import org.apache.parquet.internal.column.columnindex.ColumnIndex;
import org.apache.parquet.internal.column.columnindex.OffsetIndex;
import org.apache.parquet.io.InputFile;
import org.slf4j.Logger;

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

InputFile in = HadoopInputFile.fromPath(qualifiedPath(files.get(0)), getConf());
if (!showColumnIndex && !showOffsetIndex) {
showColumnIndex = true;
showOffsetIndex = true;
Expand All @@ -94,7 +91,7 @@ public int run() throws IOException {
rowGroupIndexSet.addAll(rowGroupIndexes);
}

try (ParquetFileReader reader = ParquetFileReader.open(in)) {
try (ParquetFileReader reader = createParquetFileReader(files.get(0))) {
boolean firstBlock = true;
int rowGroupIndex = 0;
for (BlockMetaData block : reader.getFooter().getBlocks()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ public int run() throws IOException {

String source = targets.get(0);

try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) {
try (ParquetFileReader reader = createParquetFileReader(source)) {
MessageType schema = reader.getFileMetaData().getSchema();
ColumnDescriptor descriptor = Util.descriptor(column, schema);
PrimitiveType type = Util.primitive(column, schema);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ public int run() throws IOException {

private String readFooter(InputFile inputFile) throws JsonProcessingException, IOException {
String json;
try (ParquetFileReader reader = ParquetFileReader.open(inputFile)) {
try (ParquetFileReader reader = createParquetFileReader(target)) {
ParquetMetadata footer = reader.getFooter();
ObjectMapper mapper = RawUtils.createObjectMapper();
mapper.setVisibility(PropertyAccessor.ALL, Visibility.NONE);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public int run() throws IOException {
Preconditions.checkArgument(targets.size() == 1, "Cannot process multiple Parquet files.");

String source = targets.get(0);
try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) {
try (ParquetFileReader reader = createParquetFileReader(source)) {
ParquetMetadata footer = reader.getFooter();
MessageType schema = footer.getFileMetaData().getSchema();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public int run() throws IOException {
Preconditions.checkArgument(targets.size() == 1, "Cannot process multiple Parquet files.");

String source = targets.get(0);
try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) {
try (ParquetFileReader reader = createParquetFileReader(source)) {
ParquetMetadata footer = reader.getFooter();
MessageType schema = footer.getFileMetaData().getSchema();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ public void testColumnSizeCommand() throws IOException {
@Test
public void testColumnSize() throws Exception {
String inputFile = createParquetFile();
command.setConf(conf);
Map<String, Long> columnSizeInBytes = command.getColumnSizeInBytes(new Path(inputFile));
assertEquals(columnSizeInBytes.size(), 2);
assertTrue(columnSizeInBytes.get("DocId") > columnSizeInBytes.get("Num"));
Expand Down Expand Up @@ -85,4 +86,18 @@ private String createParquetFile() throws IOException {

return file;
}

@Test
public void testColumnSizeCommandWithEncryptedFile() throws IOException {
File encryptedFile = EncryptedParquetFileTestHelper.createEncryptedParquetFile(
getTempFolder(), "encrypted_columnsize_test.parquet");

ColumnSizeCommand command = new ColumnSizeCommand(createLogger());
command.target = encryptedFile.getAbsolutePath();
command.setConf(EncryptedParquetFileTestHelper.createDecryptionConfiguration());

Assert.assertEquals(0, command.run());

encryptedFile.delete();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,177 @@
/*
* 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.parquet.cli.commands;

import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;

import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.crypto.ColumnEncryptionProperties;
import org.apache.parquet.crypto.FileEncryptionProperties;
import org.apache.parquet.crypto.ParquetCipher;
import org.apache.parquet.example.data.Group;
import org.apache.parquet.example.data.simple.SimpleGroup;
import org.apache.parquet.example.data.simple.SimpleGroupFactory;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.example.ExampleParquetWriter;
import org.apache.parquet.hadoop.example.GroupWriteSupport;
import org.apache.parquet.hadoop.metadata.ColumnPath;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.io.api.Binary;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.Types;

/**
* Utility class for creating encrypted Parquet files for testing CLI commands.
*/
public final class EncryptedParquetFileTestHelper {

// Standard test encryption keys (16 bytes for AES-128)
public static final byte[] FOOTER_KEY = {
0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10
};

public static final byte[] COLUMN_KEY_1 = {
0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11
};

public static final byte[] COLUMN_KEY_2 = {
0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11, 0x12, 0x13
};

public static final String FOOTER_KEY_HEX = "0102030405060708090a0b0c0d0e0f10";
public static final String COLUMN_KEY_1_HEX = "02030405060708090a0b0c0d0e0f1011";
public static final String COLUMN_KEY_2_HEX = "0405060708090a0b0c0d0e0f10111213";

public static final String COLUMN_KEYS_CONFIG = COLUMN_KEY_1_HEX + ":name,email;" + COLUMN_KEY_2_HEX + ":phone";

private EncryptedParquetFileTestHelper() {}

public static File createEncryptedParquetFile(File tempDir, String filename) throws IOException {
return createEncryptedParquetFile(tempDir, filename, true, true);
}

public static File createEncryptedParquetFile(
File tempDir, String filename, boolean enableBloomFilter, boolean encryptedFooter) throws IOException {

MessageType schema = Types.buildMessage()
.required(INT32)
.named("id")
.required(BINARY)
.named("name")
.required(BINARY)
.named("email")
.required(BINARY)
.named("phone")
.named("test_schema");

File file = new File(tempDir, filename);
file.deleteOnExit();

Configuration conf = new Configuration();
GroupWriteSupport.setSchema(schema, conf);

String[] encryptColumns = {"name", "email", "phone"};
FileEncryptionProperties encryptionProperties =
createFileEncryptionProperties(encryptColumns, ParquetCipher.AES_GCM_CTR_V1, encryptedFooter);

SimpleGroupFactory factory = new SimpleGroupFactory(schema);
String[] nameValues = {"test_value_1", "test_value_2", "another_test", "bloom_filter_test", "final_value"};
String[] emailValues = {
"user1@test.com", "user2@test.com", "admin@test.com", "support@test.com", "sales@test.com"
};
String[] phoneValues = {"555-0001", "555-0002", "555-0003", "555-0004", "555-0005"};

ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(new Path(file.toURI()))
.withConf(conf)
.withCompressionCodec(CompressionCodecName.UNCOMPRESSED)
.withEncryption(encryptionProperties)
.withPageSize(1024)
.withRowGroupSize(4096);

if (enableBloomFilter) {
builder.withBloomFilterEnabled("name", true)
.withBloomFilterEnabled("email", true)
.withBloomFilterEnabled("phone", true);
}

try (ParquetWriter<Group> writer = builder.build()) {
for (int i = 0; i < nameValues.length; i++) {
SimpleGroup group = (SimpleGroup) factory.newGroup();
group.add("id", i + 1);
group.add("name", Binary.fromString(nameValues[i]));
group.add("email", Binary.fromString(emailValues[i]));
group.add("phone", Binary.fromString(phoneValues[i]));
writer.write(group);
}
}

return file;
}

public static FileEncryptionProperties createFileEncryptionProperties(
String[] encryptColumns, ParquetCipher cipher, boolean footerEncryption) {

Map<String, byte[]> columnKeys = new HashMap<>();
columnKeys.put("name", COLUMN_KEY_1);
columnKeys.put("email", COLUMN_KEY_1);
columnKeys.put("phone", COLUMN_KEY_2);

Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new HashMap<>();
for (String columnPath : encryptColumns) {
ColumnPath column = ColumnPath.fromDotString(columnPath);
byte[] columnKey = columnKeys.get(columnPath);

ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder(column)
.withKey(columnKey)
.withKeyMetaData(columnPath.getBytes(StandardCharsets.UTF_8))
.build();
columnPropertyMap.put(column, columnProps);
}

FileEncryptionProperties.Builder builder = FileEncryptionProperties.builder(FOOTER_KEY)
.withFooterKeyMetadata("footkey".getBytes(StandardCharsets.UTF_8))
.withAlgorithm(cipher)
.withEncryptedColumns(columnPropertyMap);

if (!footerEncryption) {
builder.withPlaintextFooter();
}

return builder.build();
}

public static Configuration createDecryptionConfiguration() {
Configuration conf = new Configuration();
conf.set("parquet.encryption.footer.key", FOOTER_KEY_HEX);
conf.set("parquet.encryption.column.keys", COLUMN_KEYS_CONFIG);
return conf;
}

public static void setDecryptionProperties(Configuration conf) {
conf.set("parquet.encryption.footer.key", FOOTER_KEY_HEX);
conf.set("parquet.encryption.column.keys", COLUMN_KEYS_CONFIG);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,4 +34,18 @@ public void testParquetMetadataCommand() throws IOException {
command.setConf(new Configuration());
Assert.assertEquals(0, command.run());
}

@Test
public void testParquetMetadataCommandWithEncryptedFile() throws IOException {
File encryptedFile = EncryptedParquetFileTestHelper.createEncryptedParquetFile(
getTempFolder(), "encrypted_metadata_test.parquet");

ParquetMetadataCommand command = new ParquetMetadataCommand(createLogger());
command.targets = Arrays.asList(encryptedFile.getAbsolutePath());
command.setConf(EncryptedParquetFileTestHelper.createDecryptionConfiguration());

Assert.assertEquals(0, command.run());

encryptedFile.delete();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -63,4 +63,19 @@ public void testSchemaCommandOverwriteExistentFileWithoutOverwriteOption() throw
command.setConf(new Configuration());
command.run();
}

@Test
public void testSchemaCommandWithEncryptedFileParquetSchema() throws IOException {
File encryptedFile = EncryptedParquetFileTestHelper.createEncryptedParquetFile(
getTempFolder(), "encrypted_schema_parquet_test.parquet");

SchemaCommand command = new SchemaCommand(createLogger());
command.targets = Arrays.asList(encryptedFile.getAbsolutePath());
command.parquetSchema = true;
command.setConf(EncryptedParquetFileTestHelper.createDecryptionConfiguration());

Assert.assertEquals(0, command.run());

encryptedFile.delete();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,4 +34,18 @@ public void testShowColumnIndexCommand() throws IOException {
command.setConf(new Configuration());
Assert.assertEquals(0, command.run());
}

@Test
public void testShowColumnIndexCommandWithEncryptedFile() throws IOException {
File encryptedFile = EncryptedParquetFileTestHelper.createEncryptedParquetFile(
getTempFolder(), "encrypted_columnindex_test.parquet");

ShowColumnIndexCommand command = new ShowColumnIndexCommand(createLogger());
command.files = Arrays.asList(encryptedFile.getAbsolutePath());
command.setConf(EncryptedParquetFileTestHelper.createDecryptionConfiguration());

Assert.assertEquals(0, command.run());

encryptedFile.delete();
}
}
Loading
Loading