From d494d6b75170ca23cca07ad2054c7048ee89eb39 Mon Sep 17 00:00:00 2001 From: arnavb Date: Mon, 25 Aug 2025 15:43:25 +0000 Subject: [PATCH 1/7] update --- .../java/org/apache/parquet/cli/Main.java | 1 + .../cli/commands/EncryptionInfoCommand.java | 204 ++++++++++++ .../commands/TestEncryptionInfoCommand.java | 296 ++++++++++++++++++ 3 files changed, 501 insertions(+) create mode 100644 parquet-cli/src/main/java/org/apache/parquet/cli/commands/EncryptionInfoCommand.java create mode 100644 parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java b/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java index 62940054e6..5ff999097d 100644 --- a/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java @@ -111,6 +111,7 @@ public class Main extends Configured implements Tool { jc.addCommand("rewrite", new RewriteCommand(console)); jc.addCommand("size-stats", new ShowSizeStatisticsCommand(console)); jc.addCommand("geospatial-stats", new ShowGeospatialStatisticsCommand(console)); + jc.addCommand("encryption-info", new org.apache.parquet.cli.commands.EncryptionInfoCommand(console)); } @Override diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/EncryptionInfoCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/EncryptionInfoCommand.java new file mode 100644 index 0000000000..1a72e6febe --- /dev/null +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/EncryptionInfoCommand.java @@ -0,0 +1,204 @@ +/* + * 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.cli.Util.encodingsAsString; +import static org.apache.parquet.cli.Util.primitive; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import javax.annotation.Nullable; +import org.apache.commons.text.TextStringBuilder; +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; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.FileMetaData.EncryptionType; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; + +/** + * Prints encryption information for a Parquet file. + *

+ * The command never attempts to decrypt data. If the footer is encrypted it + * simply reports that fact. + */ +@Parameters(commandDescription = "Print Parquet file encryption details") +public class EncryptionInfoCommand extends BaseCommand { + + public EncryptionInfoCommand(Logger console) { + super(console); + } + + @Parameter(description = "", required = true) + List targets; + + @Override + public int run() throws IOException { + Preconditions.checkArgument(targets.size() == 1, "Exactly one Parquet file must be specified."); + String source = targets.get(0); + + ParquetMetadata footer = + ParquetFileReader.readFooter(getConf(), qualifiedPath(source), ParquetMetadataConverter.NO_FILTER); + + FileMetaData meta = footer.getFileMetaData(); + console.info("File: {}", source); + console.info("Encryption type: {}", meta.getEncryptionType()); + + if (meta.getEncryptionType() == EncryptionType.ENCRYPTED_FOOTER) { + console.info("Footer is encrypted – column list unavailable without keys."); + return 0; + } + + ColumnEncryptionInfo encryptionInfo = analyzeColumnEncryption(footer); + + console.info("Column encryption summary:"); + console.info(" Total columns: {}", encryptionInfo.getTotalColumns()); + console.info( + " Encrypted columns: {}", encryptionInfo.getEncryptedColumns().size()); + console.info( + " Unencrypted columns: {}", + encryptionInfo.getUnencryptedColumns().size()); + + if (!encryptionInfo.getEncryptedColumns().isEmpty()) { + console.info(" Encrypted columns:"); + for (String col : encryptionInfo.getEncryptedColumns()) { + console.info(" {}", col); + } + } + + if (!footer.getBlocks().isEmpty()) { + console.info("\nColumns:"); + listColumnEncryptionStatus(console, footer); + } + + return 0; + } + + @Override + public java.util.List getExamples() { + return java.util.Collections.singletonList("encryption-info sample.parquet"); + } + + private void listColumnEncryptionStatus(Logger console, ParquetMetadata footer) { + BlockMetaData firstRowGroup = footer.getBlocks().get(0); + + int maxColumnWidth = + maxSize(Iterables.transform(firstRowGroup.getColumns(), new Function() { + @Override + public String apply(@Nullable ColumnChunkMetaData input) { + return input == null ? "" : input.getPath().toDotString(); + } + })); + + console.info( + String.format("%-" + maxColumnWidth + "s %-9s %-9s %s", "Column", "Encrypted", "Type", "Encodings")); + console.info(new TextStringBuilder(maxColumnWidth + 40) + .appendPadding(maxColumnWidth + 40, '-') + .toString()); + + MessageType schema = footer.getFileMetaData().getSchema(); + + for (ColumnChunkMetaData column : firstRowGroup.getColumns()) { + String name = column.getPath().toDotString(); + String status = column.isEncrypted() ? "ENCRYPTED" : "-"; + + PrimitiveType ptype = primitive(schema, column.getPath().toArray()); + String typeStr = ptype.getPrimitiveTypeName().name(); + + String enc = encodingsAsString( + column.getEncodings(), + schema.getColumnDescription(column.getPath().toArray())); + + console.info(String.format("%-" + maxColumnWidth + "s %-9s %-9s %s", name, status, typeStr, enc)); + } + } + + private int maxSize(Iterable strings) { + int size = 0; + for (String s : strings) { + size = Math.max(size, s.length()); + } + return size; + } + + /** Visible for test */ + static Set findEncryptedColumns(ParquetMetadata footer) { + Set cols = new TreeSet<>(); + for (BlockMetaData rg : footer.getBlocks()) { + for (ColumnChunkMetaData c : rg.getColumns()) { + if (c.isEncrypted()) { + cols.add(c.getPath().toDotString()); + } + } + } + return cols; + } + + private ColumnEncryptionInfo analyzeColumnEncryption(ParquetMetadata footer) { + Set encryptedColumns = new TreeSet<>(); + Set unencryptedColumns = new TreeSet<>(); + + for (BlockMetaData rg : footer.getBlocks()) { + for (ColumnChunkMetaData c : rg.getColumns()) { + String columnName = c.getPath().toDotString(); + if (c.isEncrypted()) { + encryptedColumns.add(columnName); + } else { + unencryptedColumns.add(columnName); + } + } + } + + return new ColumnEncryptionInfo(encryptedColumns, unencryptedColumns); + } + + private static class ColumnEncryptionInfo { + private final Set encryptedColumns; + private final Set unencryptedColumns; + + public ColumnEncryptionInfo(Set encryptedColumns, Set unencryptedColumns) { + this.encryptedColumns = encryptedColumns; + this.unencryptedColumns = unencryptedColumns; + } + + public Set getEncryptedColumns() { + return encryptedColumns; + } + + public Set getUnencryptedColumns() { + return unencryptedColumns; + } + + public int getTotalColumns() { + return encryptedColumns.size() + unencryptedColumns.size(); + } + } +} diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java new file mode 100644 index 0000000000..e58ecc3540 --- /dev/null +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java @@ -0,0 +1,296 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +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.ParquetReadOptions; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.crypto.ColumnEncryptionProperties; +import org.apache.parquet.crypto.DecryptionKeyRetriever; +import org.apache.parquet.crypto.FileDecryptionProperties; +import org.apache.parquet.crypto.FileEncryptionProperties; +import org.apache.parquet.crypto.KeyAccessDeniedException; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroupFactory; +import org.apache.parquet.hadoop.ParquetFileReader; +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.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Types; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import org.slf4j.Logger; + +public class TestEncryptionInfoCommand extends ParquetFileTest { + + private static final byte[] FOOTER_KEY = "0123456789012345".getBytes(); + private static final byte[] COLUMN_KEY = "1234567890123450".getBytes(); + private static final String FOOTER_KEY_ID = "kf"; + private static final String COLUMN_KEY_ID = "kc"; + + private File unencryptedFile; + private File columnEncryptedFile; + private File footerEncryptedFile; + + @Before + public void setUp() throws IOException { + super.setUp(); + createTestFiles(); + } + + private void createTestFiles() throws IOException { + unencryptedFile = createParquetFile("unencrypted.parquet", null); + + FileEncryptionProperties columnEncProps = createColumnEncryptionProperties(); + columnEncryptedFile = createParquetFile("column_encrypted.parquet", columnEncProps); + + FileEncryptionProperties footerEncProps = createFooterEncryptionProperties(); + footerEncryptedFile = createParquetFile("footer_encrypted.parquet", footerEncProps); + } + + private FileEncryptionProperties createColumnEncryptionProperties() { + String columnKeyMetadata = "{\"key\":\"" + COLUMN_KEY_ID + "\"}"; + String footerKeyMetadata = "{\"key\":\"" + FOOTER_KEY_ID + "\"}"; + + ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder("ssn") + .withKey(COLUMN_KEY) + .withKeyMetaData(columnKeyMetadata.getBytes(StandardCharsets.UTF_8)) + .build(); + + Map columnMap = new HashMap<>(); + columnMap.put(columnProps.getPath(), columnProps); + + return FileEncryptionProperties.builder(FOOTER_KEY) + .withFooterKeyMetadata(footerKeyMetadata.getBytes(StandardCharsets.UTF_8)) + .withEncryptedColumns(columnMap) + .build(); + } + + private FileEncryptionProperties createFooterEncryptionProperties() { + // Use JSON metadata format that PropertiesDrivenCryptoFactory expects + String footerKeyMetadata = "{\"key\":\"" + FOOTER_KEY_ID + "\"}"; + + return FileEncryptionProperties.builder(FOOTER_KEY) + .withFooterKeyMetadata(footerKeyMetadata.getBytes(StandardCharsets.UTF_8)) + .build(); + } + + private FileDecryptionProperties createDecryptionProperties() { + DecryptionKeyRetriever keyRetriever = new DecryptionKeyRetriever() { + @Override + public byte[] getKey(byte[] keyMetaData) throws KeyAccessDeniedException, ParquetCryptoRuntimeException { + String keyID = new String(keyMetaData); + if (FOOTER_KEY_ID.equals(keyID)) { + return FOOTER_KEY; + } + if (COLUMN_KEY_ID.equals(keyID)) { + return COLUMN_KEY; + } + return null; + } + }; + + return FileDecryptionProperties.builder().withKeyRetriever(keyRetriever).build(); + } + + private File createParquetFile(String filename, FileEncryptionProperties encryptionProps) throws IOException { + File file = new File(getTempFolder(), filename); + Path fsPath = new Path(file.getPath()); + Configuration conf = new Configuration(); + + MessageType schema = Types.buildMessage() + .required(org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32) + .named("id") + .required(org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY) + .named("ssn") + .required(org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY) + .named("name") + .named("schema"); + + SimpleGroupFactory fact = new SimpleGroupFactory(schema); + GroupWriteSupport.setSchema(schema, conf); + + ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(fsPath) + .withConf(conf) + .withWriteMode(org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE) + .withCompressionCodec(CompressionCodecName.UNCOMPRESSED) + .withRowGroupSize(1024) + .withPageSize(1024) + .withDictionaryPageSize(512) + .withDictionaryEncoding(true) + .withValidation(false) + .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_2_0); + + if (encryptionProps != null) { + builder.withEncryption(encryptionProps); + } + + try (ParquetWriter writer = builder.build()) { + for (int i = 0; i < 5; i++) { + writer.write(fact.newGroup() + .append("id", i) + .append("ssn", "123-45-" + String.format("%04d", i)) + .append("name", "User" + i)); + } + } + + return file; + } + + private static void enableCliDecryptionKeys() { + System.setProperty( + "parquet.crypto.factory.class", "org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory"); + System.setProperty( + "parquet.encryption.key.list", + FOOTER_KEY_ID + ":" + bytesToHex(FOOTER_KEY) + "," + COLUMN_KEY_ID + ":" + bytesToHex(COLUMN_KEY)); + } + + private static String bytesToHex(byte[] bytes) { + StringBuilder sb = new StringBuilder(); + for (byte b : bytes) { + sb.append(String.format("%02x", b)); + } + return sb.toString(); + } + + @Test + public void testUnencryptedFile() throws IOException { + EncryptionInfoCommand command = new EncryptionInfoCommand(createLogger()); + command.targets = java.util.Arrays.asList(unencryptedFile.getAbsolutePath()); + command.setConf(new Configuration()); + + int result = command.run(); + assertEquals(0, result); + } + + @Test + public void testColumnEncryptedFile() throws IOException { + EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger()); + cmd.targets = java.util.Arrays.asList(columnEncryptedFile.getAbsolutePath()); + cmd.setConf(new Configuration()); + + try { + cmd.run(); + fail("Expected ParquetCryptoRuntimeException for encrypted file without keys"); + } catch (org.apache.parquet.crypto.ParquetCryptoRuntimeException e) { + assertTrue(e.getMessage().contains("encrypted") || e.getMessage().contains("key")); + } + } + + @Test + public void testFooterEncryptedFile() throws IOException { + EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger()); + cmd.targets = java.util.Arrays.asList(footerEncryptedFile.getAbsolutePath()); + cmd.setConf(new Configuration()); + + try { + cmd.run(); + fail("Expected ParquetCryptoRuntimeException for encrypted file without keys"); + } catch (org.apache.parquet.crypto.ParquetCryptoRuntimeException e) { + assertTrue(e.getMessage().contains("encrypted") || e.getMessage().contains("key")); + } + } + + @Test + public void testCommandWorksWithRealFile() throws IOException { + File file = parquetFile(); + EncryptionInfoCommand command = new EncryptionInfoCommand(createLogger()); + command.targets = java.util.Arrays.asList(file.getAbsolutePath()); + command.setConf(new Configuration()); + + int result = command.run(); + assertEquals(0, result); + } + + @Test + public void testCommandExists() { + EncryptionInfoCommand cmd = new EncryptionInfoCommand(LoggerFactory.getLogger(TestEncryptionInfoCommand.class)); + } + + @Test + public void testColumnEncryptedFileWithKeys() throws IOException { + assertTrue(columnEncryptedFile.exists()); + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + PrintStream ps = new PrintStream(out, true, StandardCharsets.UTF_8); + Logger logger = createLogger(ps); + + EncryptionInfoCommand cmd = new EncryptionInfoCommand(logger); + cmd.targets = java.util.Arrays.asList(columnEncryptedFile.getAbsolutePath()); + cmd.setConf(new Configuration()); + + int rc = cmd.run(); + assertEquals(0, rc); + + String output = out.toString(StandardCharsets.UTF_8); + + assertTrue(output.contains("Encryption type: PLAINTEXT_FOOTER")); + assertTrue(output.contains("ssn ENCRYPTED")); + assertTrue(output.contains("id -")); + } + + @Test + public void testFooterEncryptedFileWithKeys() throws IOException { + FileDecryptionProperties decryptionProps = createDecryptionProperties(); + Configuration verifyConf = new Configuration(); + + ParquetReadOptions readOptions = ParquetReadOptions.builder() + .withDecryption(decryptionProps) + .build(); + InputFile inputFile = HadoopInputFile.fromPath(new Path(footerEncryptedFile.getPath()), verifyConf); + + try { + ParquetFileReader.open(inputFile, readOptions); + fail("Expected ParquetCryptoRuntimeException for footer encrypted file without keys"); + } catch (ParquetCryptoRuntimeException e) { + } + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Logger logger = createLogger(new PrintStream(out)); + EncryptionInfoCommand cmd = new EncryptionInfoCommand(logger); + cmd.targets = java.util.Arrays.asList(footerEncryptedFile.getAbsolutePath()); + cmd.setConf(new Configuration()); + + int rc = cmd.run(); + assertEquals(0, rc); + String output = out.toString(); + assertTrue(output.contains("Encryption type: ENCRYPTED_FOOTER")); + assertTrue(output.contains("Footer is encrypted")); + } +} From 58ea6506c1e4bb16736ac2b764154219b5650644 Mon Sep 17 00:00:00 2001 From: arnavb Date: Mon, 25 Aug 2025 17:48:14 +0000 Subject: [PATCH 2/7] update --- .../cli/commands/TestEncryptionInfoCommand.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java index e58ecc3540..d1a9525258 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java @@ -22,8 +22,10 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; +import java.io.PrintStream; import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; @@ -51,11 +53,8 @@ import org.apache.parquet.schema.Types; import org.junit.Before; import org.junit.Test; -import org.slf4j.LoggerFactory; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestEncryptionInfoCommand extends ParquetFileTest { @@ -270,9 +269,8 @@ public void testFooterEncryptedFileWithKeys() throws IOException { FileDecryptionProperties decryptionProps = createDecryptionProperties(); Configuration verifyConf = new Configuration(); - ParquetReadOptions readOptions = ParquetReadOptions.builder() - .withDecryption(decryptionProps) - .build(); + ParquetReadOptions readOptions = + ParquetReadOptions.builder().withDecryption(decryptionProps).build(); InputFile inputFile = HadoopInputFile.fromPath(new Path(footerEncryptedFile.getPath()), verifyConf); try { From 6a16090b2cff5c6586cee2ea15b5088f6f452b55 Mon Sep 17 00:00:00 2001 From: arnavb Date: Mon, 25 Aug 2025 22:24:46 +0000 Subject: [PATCH 3/7] update --- .../commands/TestEncryptionInfoCommand.java | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java index d1a9525258..02247d0ea6 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java @@ -27,8 +27,10 @@ import java.io.IOException; import java.io.PrintStream; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.ParquetReadOptions; @@ -47,6 +49,8 @@ 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.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.metadata.ParquetMetadataConverter; import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.io.InputFile; import org.apache.parquet.schema.MessageType; @@ -246,22 +250,19 @@ public void testCommandExists() { public void testColumnEncryptedFileWithKeys() throws IOException { assertTrue(columnEncryptedFile.exists()); - ByteArrayOutputStream out = new ByteArrayOutputStream(); - PrintStream ps = new PrintStream(out, true, StandardCharsets.UTF_8); - Logger logger = createLogger(ps); - - EncryptionInfoCommand cmd = new EncryptionInfoCommand(logger); + EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger()); cmd.targets = java.util.Arrays.asList(columnEncryptedFile.getAbsolutePath()); cmd.setConf(new Configuration()); int rc = cmd.run(); assertEquals(0, rc); - String output = out.toString(StandardCharsets.UTF_8); - - assertTrue(output.contains("Encryption type: PLAINTEXT_FOOTER")); - assertTrue(output.contains("ssn ENCRYPTED")); - assertTrue(output.contains("id -")); + ParquetMetadata footer = ParquetFileReader.readFooter( + new Configuration(), + new Path(columnEncryptedFile.getAbsolutePath()), + ParquetMetadataConverter.NO_FILTER); + Set encrypted = EncryptionInfoCommand.findEncryptedColumns(footer); + assertEquals(Collections.singleton("ssn"), encrypted); } @Test From 58801a835930b5771546e014cac18b4cd23be880 Mon Sep 17 00:00:00 2001 From: arnavb Date: Tue, 26 Aug 2025 06:20:48 +0000 Subject: [PATCH 4/7] lint --- .../test/java/org/apache/parquet/cli/commands/FileTest.java | 4 ++++ .../parquet/cli/commands/TestEncryptionInfoCommand.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/FileTest.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/FileTest.java index 6e031112ff..a1172b7695 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/FileTest.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/FileTest.java @@ -58,6 +58,10 @@ protected static Logger createLogger() { return console; } + protected static Logger createLogger(java.io.PrintStream out) { + return createLogger(); + } + @FunctionalInterface public interface ThrowableBiConsumer { void accept(T t, U u) throws Exception; diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java index 02247d0ea6..8dc3d843f4 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java @@ -43,6 +43,7 @@ import org.apache.parquet.crypto.ParquetCryptoRuntimeException; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroupFactory; +import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.example.ExampleParquetWriter; @@ -50,7 +51,6 @@ import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.hadoop.metadata.ParquetMetadataConverter; import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.io.InputFile; import org.apache.parquet.schema.MessageType; From d3075318ffabb236dc40c0aa390d07193d16e884 Mon Sep 17 00:00:00 2001 From: arnavb Date: Tue, 26 Aug 2025 11:08:08 +0000 Subject: [PATCH 5/7] lint --- .../cli/commands/TestEncryptionInfoCommand.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java index 8dc3d843f4..2f162095dc 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java @@ -250,6 +250,8 @@ public void testCommandExists() { public void testColumnEncryptedFileWithKeys() throws IOException { assertTrue(columnEncryptedFile.exists()); + enableCliDecryptionKeys(); + EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger()); cmd.targets = java.util.Arrays.asList(columnEncryptedFile.getAbsolutePath()); cmd.setConf(new Configuration()); @@ -257,10 +259,13 @@ public void testColumnEncryptedFileWithKeys() throws IOException { int rc = cmd.run(); assertEquals(0, rc); - ParquetMetadata footer = ParquetFileReader.readFooter( - new Configuration(), - new Path(columnEncryptedFile.getAbsolutePath()), - ParquetMetadataConverter.NO_FILTER); + FileDecryptionProperties decryptionProps = createDecryptionProperties(); + ParquetReadOptions options = ParquetReadOptions.builder().withDecryption(decryptionProps).build(); + InputFile inputFile = HadoopInputFile.fromPath(new Path(columnEncryptedFile.getAbsolutePath()), new Configuration()); + ParquetMetadata footer; + try (ParquetFileReader reader = ParquetFileReader.open(inputFile, options)) { + footer = reader.getFooter(); + } Set encrypted = EncryptionInfoCommand.findEncryptedColumns(footer); assertEquals(Collections.singleton("ssn"), encrypted); } @@ -281,6 +286,7 @@ public void testFooterEncryptedFileWithKeys() throws IOException { } ByteArrayOutputStream out = new ByteArrayOutputStream(); + enableCliDecryptionKeys(); Logger logger = createLogger(new PrintStream(out)); EncryptionInfoCommand cmd = new EncryptionInfoCommand(logger); cmd.targets = java.util.Arrays.asList(footerEncryptedFile.getAbsolutePath()); From 922217161ca3c8512856c9cf84cb6d719aaacd0a Mon Sep 17 00:00:00 2001 From: arnavb Date: Tue, 26 Aug 2025 11:10:27 +0000 Subject: [PATCH 6/7] lint --- .../parquet/cli/commands/TestEncryptionInfoCommand.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java index 2f162095dc..21a6fe5088 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java @@ -43,7 +43,6 @@ import org.apache.parquet.crypto.ParquetCryptoRuntimeException; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroupFactory; -import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.example.ExampleParquetWriter; @@ -260,8 +259,10 @@ public void testColumnEncryptedFileWithKeys() throws IOException { assertEquals(0, rc); FileDecryptionProperties decryptionProps = createDecryptionProperties(); - ParquetReadOptions options = ParquetReadOptions.builder().withDecryption(decryptionProps).build(); - InputFile inputFile = HadoopInputFile.fromPath(new Path(columnEncryptedFile.getAbsolutePath()), new Configuration()); + ParquetReadOptions options = + ParquetReadOptions.builder().withDecryption(decryptionProps).build(); + InputFile inputFile = + HadoopInputFile.fromPath(new Path(columnEncryptedFile.getAbsolutePath()), new Configuration()); ParquetMetadata footer; try (ParquetFileReader reader = ParquetFileReader.open(inputFile, options)) { footer = reader.getFooter(); From 9fe010270efbd8e479db66b5f1f97dcd0dd6a2cd Mon Sep 17 00:00:00 2001 From: arnavb Date: Wed, 27 Aug 2025 05:41:37 +0000 Subject: [PATCH 7/7] update --- parquet-cli/pom.xml | 8 + .../commands/TestEncryptionInfoCommand.java | 257 +++++++++--------- 2 files changed, 135 insertions(+), 130 deletions(-) diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 6bd0b5e7d5..4701f1e366 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -105,6 +105,14 @@ parquet-hadoop ${project.version} + + + org.apache.parquet + parquet-hadoop + tests + ${project.version} + test + org.apache.avro avro diff --git a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java index 21a6fe5088..d15446c154 100644 --- a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java +++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java @@ -22,54 +22,65 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; -import java.io.PrintStream; import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Base64; import java.util.Collections; -import java.util.HashMap; -import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.column.ParquetProperties; -import org.apache.parquet.crypto.ColumnEncryptionProperties; -import org.apache.parquet.crypto.DecryptionKeyRetriever; +import org.apache.parquet.crypto.DecryptionPropertiesFactory; +import org.apache.parquet.crypto.EncryptionPropertiesFactory; import org.apache.parquet.crypto.FileDecryptionProperties; import org.apache.parquet.crypto.FileEncryptionProperties; -import org.apache.parquet.crypto.KeyAccessDeniedException; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroupFactory; import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; 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.hadoop.metadata.ParquetMetadata; import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.io.InputFile; import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Types; import org.junit.Before; import org.junit.Test; -import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * This test checks the CLI functionality for encryption info. + */ public class TestEncryptionInfoCommand extends ParquetFileTest { - private static final byte[] FOOTER_KEY = "0123456789012345".getBytes(); - private static final byte[] COLUMN_KEY = "1234567890123450".getBytes(); + private static final byte[] FOOTER_KEY = "0123456789012345".getBytes(StandardCharsets.UTF_8); + private static final byte[] COLUMN_KEY = "1234567890123450".getBytes(StandardCharsets.UTF_8); + private static final String FOOTER_KEY_ID = "kf"; private static final String COLUMN_KEY_ID = "kc"; + private static final String FACTORY_CLASS = "org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory"; + private static final String MOCK_KMS_CLASS = "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS"; + private File unencryptedFile; private File columnEncryptedFile; private File footerEncryptedFile; + private enum EncMode { + NONE, // No encryption + COLUMN_AND_FOOTER, // Both column and footer encrypted + COLUMN_PLAINTEXT_FOOTER // Column encrypted, footer plaintext (legacy compatible) + } + @Before public void setUp() throws IOException { super.setUp(); @@ -77,71 +88,35 @@ public void setUp() throws IOException { } private void createTestFiles() throws IOException { - unencryptedFile = createParquetFile("unencrypted.parquet", null); - - FileEncryptionProperties columnEncProps = createColumnEncryptionProperties(); - columnEncryptedFile = createParquetFile("column_encrypted.parquet", columnEncProps); - - FileEncryptionProperties footerEncProps = createFooterEncryptionProperties(); - footerEncryptedFile = createParquetFile("footer_encrypted.parquet", footerEncProps); + unencryptedFile = createParquetFile("unencrypted.parquet", EncMode.NONE); + columnEncryptedFile = createParquetFile("column_encrypted.parquet", EncMode.COLUMN_AND_FOOTER); + footerEncryptedFile = createParquetFile("footer_encrypted.parquet", EncMode.COLUMN_PLAINTEXT_FOOTER); } - private FileEncryptionProperties createColumnEncryptionProperties() { - String columnKeyMetadata = "{\"key\":\"" + COLUMN_KEY_ID + "\"}"; - String footerKeyMetadata = "{\"key\":\"" + FOOTER_KEY_ID + "\"}"; - - ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder("ssn") - .withKey(COLUMN_KEY) - .withKeyMetaData(columnKeyMetadata.getBytes(StandardCharsets.UTF_8)) - .build(); - - Map columnMap = new HashMap<>(); - columnMap.put(columnProps.getPath(), columnProps); - - return FileEncryptionProperties.builder(FOOTER_KEY) - .withFooterKeyMetadata(footerKeyMetadata.getBytes(StandardCharsets.UTF_8)) - .withEncryptedColumns(columnMap) - .build(); - } - - private FileEncryptionProperties createFooterEncryptionProperties() { - // Use JSON metadata format that PropertiesDrivenCryptoFactory expects - String footerKeyMetadata = "{\"key\":\"" + FOOTER_KEY_ID + "\"}"; + private File createParquetFile(String filename, EncMode mode) throws IOException { + File file = new File(getTempFolder(), filename); + Path fsPath = new Path(file.getPath()); - return FileEncryptionProperties.builder(FOOTER_KEY) - .withFooterKeyMetadata(footerKeyMetadata.getBytes(StandardCharsets.UTF_8)) - .build(); - } + Configuration conf = getHadoopConfiguration(mode); + FileEncryptionProperties fileEncryptionProperties = null; - private FileDecryptionProperties createDecryptionProperties() { - DecryptionKeyRetriever keyRetriever = new DecryptionKeyRetriever() { - @Override - public byte[] getKey(byte[] keyMetaData) throws KeyAccessDeniedException, ParquetCryptoRuntimeException { - String keyID = new String(keyMetaData); - if (FOOTER_KEY_ID.equals(keyID)) { - return FOOTER_KEY; - } - if (COLUMN_KEY_ID.equals(keyID)) { - return COLUMN_KEY; - } - return null; + try { + if (null == conf) { + conf = new Configuration(); + } else { + EncryptionPropertiesFactory cryptoFactory = EncryptionPropertiesFactory.loadFactory(conf); + fileEncryptionProperties = cryptoFactory.getFileEncryptionProperties(conf, fsPath, null); } - }; - - return FileDecryptionProperties.builder().withKeyRetriever(keyRetriever).build(); - } - - private File createParquetFile(String filename, FileEncryptionProperties encryptionProps) throws IOException { - File file = new File(getTempFolder(), filename); - Path fsPath = new Path(file.getPath()); - Configuration conf = new Configuration(); + } catch (Exception e) { + throw new IOException("Failed to create encryption properties for " + filename, e); + } MessageType schema = Types.buildMessage() - .required(org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32) + .required(PrimitiveType.PrimitiveTypeName.INT32) .named("id") - .required(org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY) + .required(PrimitiveType.PrimitiveTypeName.BINARY) .named("ssn") - .required(org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY) + .required(PrimitiveType.PrimitiveTypeName.BINARY) .named("name") .named("schema"); @@ -150,18 +125,15 @@ private File createParquetFile(String filename, FileEncryptionProperties encrypt ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(fsPath) .withConf(conf) - .withWriteMode(org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE) + .withWriteMode(ParquetFileWriter.Mode.OVERWRITE) .withCompressionCodec(CompressionCodecName.UNCOMPRESSED) .withRowGroupSize(1024) .withPageSize(1024) .withDictionaryPageSize(512) .withDictionaryEncoding(true) .withValidation(false) - .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_2_0); - - if (encryptionProps != null) { - builder.withEncryption(encryptionProps); - } + .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_2_0) + .withEncryption(fileEncryptionProperties); try (ParquetWriter writer = builder.build()) { for (int i = 0; i < 5; i++) { @@ -175,26 +147,51 @@ private File createParquetFile(String filename, FileEncryptionProperties encrypt return file; } - private static void enableCliDecryptionKeys() { - System.setProperty( - "parquet.crypto.factory.class", "org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory"); - System.setProperty( + private Configuration getHadoopConfiguration(EncMode mode) { + if (mode == EncMode.NONE) { + return null; + } + + Configuration conf = new Configuration(); + + conf.set("parquet.crypto.factory.class", FACTORY_CLASS); + conf.set("parquet.encryption.kms.client.class", MOCK_KMS_CLASS); + conf.set( "parquet.encryption.key.list", - FOOTER_KEY_ID + ":" + bytesToHex(FOOTER_KEY) + "," + COLUMN_KEY_ID + ":" + bytesToHex(COLUMN_KEY)); + FOOTER_KEY_ID + ":" + bytesToBase64(FOOTER_KEY) + ", " + COLUMN_KEY_ID + ":" + + bytesToBase64(COLUMN_KEY)); + + if (mode == EncMode.COLUMN_AND_FOOTER) { + conf.set(PropertiesDrivenCryptoFactory.COLUMN_KEYS_PROPERTY_NAME, COLUMN_KEY_ID + ":ssn"); + conf.set(PropertiesDrivenCryptoFactory.FOOTER_KEY_PROPERTY_NAME, FOOTER_KEY_ID); + } else if (mode == EncMode.COLUMN_PLAINTEXT_FOOTER) { + conf.set(PropertiesDrivenCryptoFactory.COLUMN_KEYS_PROPERTY_NAME, COLUMN_KEY_ID + ":ssn"); + conf.set(PropertiesDrivenCryptoFactory.FOOTER_KEY_PROPERTY_NAME, FOOTER_KEY_ID); + conf.setBoolean(PropertiesDrivenCryptoFactory.PLAINTEXT_FOOTER_PROPERTY_NAME, true); + } + + return conf; } - private static String bytesToHex(byte[] bytes) { - StringBuilder sb = new StringBuilder(); - for (byte b : bytes) { - sb.append(String.format("%02x", b)); - } - return sb.toString(); + private Configuration getDecryptionConfiguration() { + Configuration conf = new Configuration(); + conf.set("parquet.crypto.factory.class", FACTORY_CLASS); + conf.set("parquet.encryption.kms.client.class", MOCK_KMS_CLASS); + conf.set( + "parquet.encryption.key.list", + FOOTER_KEY_ID + ":" + bytesToBase64(FOOTER_KEY) + ", " + COLUMN_KEY_ID + ":" + + bytesToBase64(COLUMN_KEY)); + return conf; + } + + private static String bytesToBase64(byte[] bytes) { + return Base64.getEncoder().encodeToString(bytes); } @Test public void testUnencryptedFile() throws IOException { EncryptionInfoCommand command = new EncryptionInfoCommand(createLogger()); - command.targets = java.util.Arrays.asList(unencryptedFile.getAbsolutePath()); + command.targets = Arrays.asList(unencryptedFile.getAbsolutePath()); command.setConf(new Configuration()); int result = command.run(); @@ -204,36 +201,33 @@ public void testUnencryptedFile() throws IOException { @Test public void testColumnEncryptedFile() throws IOException { EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger()); - cmd.targets = java.util.Arrays.asList(columnEncryptedFile.getAbsolutePath()); + cmd.targets = Arrays.asList(columnEncryptedFile.getAbsolutePath()); cmd.setConf(new Configuration()); try { cmd.run(); fail("Expected ParquetCryptoRuntimeException for encrypted file without keys"); - } catch (org.apache.parquet.crypto.ParquetCryptoRuntimeException e) { - assertTrue(e.getMessage().contains("encrypted") || e.getMessage().contains("key")); + } catch (ParquetCryptoRuntimeException e) { + assertTrue(e.getMessage().toLowerCase().contains("encrypted") + || e.getMessage().toLowerCase().contains("key")); } } @Test public void testFooterEncryptedFile() throws IOException { EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger()); - cmd.targets = java.util.Arrays.asList(footerEncryptedFile.getAbsolutePath()); - cmd.setConf(new Configuration()); + cmd.targets = Arrays.asList(footerEncryptedFile.getAbsolutePath()); + cmd.setConf(getDecryptionConfiguration()); - try { - cmd.run(); - fail("Expected ParquetCryptoRuntimeException for encrypted file without keys"); - } catch (org.apache.parquet.crypto.ParquetCryptoRuntimeException e) { - assertTrue(e.getMessage().contains("encrypted") || e.getMessage().contains("key")); - } + int result = cmd.run(); + assertEquals(0, result); } @Test public void testCommandWorksWithRealFile() throws IOException { File file = parquetFile(); EncryptionInfoCommand command = new EncryptionInfoCommand(createLogger()); - command.targets = java.util.Arrays.asList(file.getAbsolutePath()); + command.targets = Arrays.asList(file.getAbsolutePath()); command.setConf(new Configuration()); int result = command.run(); @@ -242,29 +236,33 @@ public void testCommandWorksWithRealFile() throws IOException { @Test public void testCommandExists() { - EncryptionInfoCommand cmd = new EncryptionInfoCommand(LoggerFactory.getLogger(TestEncryptionInfoCommand.class)); + new EncryptionInfoCommand(LoggerFactory.getLogger(TestEncryptionInfoCommand.class)); } @Test public void testColumnEncryptedFileWithKeys() throws IOException { assertTrue(columnEncryptedFile.exists()); - enableCliDecryptionKeys(); - EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger()); - cmd.targets = java.util.Arrays.asList(columnEncryptedFile.getAbsolutePath()); - cmd.setConf(new Configuration()); - + cmd.targets = Arrays.asList(columnEncryptedFile.getAbsolutePath()); + cmd.setConf(getDecryptionConfiguration()); int rc = cmd.run(); assertEquals(0, rc); - FileDecryptionProperties decryptionProps = createDecryptionProperties(); - ParquetReadOptions options = - ParquetReadOptions.builder().withDecryption(decryptionProps).build(); - InputFile inputFile = - HadoopInputFile.fromPath(new Path(columnEncryptedFile.getAbsolutePath()), new Configuration()); + Configuration hadoopConfig = getDecryptionConfiguration(); + FileDecryptionProperties fileDecryptionProperties = null; + + DecryptionPropertiesFactory cryptoFactory = DecryptionPropertiesFactory.loadFactory(hadoopConfig); + fileDecryptionProperties = cryptoFactory.getFileDecryptionProperties( + hadoopConfig, new Path(columnEncryptedFile.getAbsolutePath())); + + InputFile inputFile = HadoopInputFile.fromPath(new Path(columnEncryptedFile.getAbsolutePath()), hadoopConfig); ParquetMetadata footer; - try (ParquetFileReader reader = ParquetFileReader.open(inputFile, options)) { + try (ParquetFileReader reader = ParquetFileReader.open( + inputFile, + ParquetReadOptions.builder() + .withDecryption(fileDecryptionProperties) + .build())) { footer = reader.getFooter(); } Set encrypted = EncryptionInfoCommand.findEncryptedColumns(footer); @@ -273,30 +271,29 @@ public void testColumnEncryptedFileWithKeys() throws IOException { @Test public void testFooterEncryptedFileWithKeys() throws IOException { - FileDecryptionProperties decryptionProps = createDecryptionProperties(); - Configuration verifyConf = new Configuration(); + withLogger((logger, events) -> { + EncryptionInfoCommand cmd = new EncryptionInfoCommand(logger); + cmd.targets = Arrays.asList(footerEncryptedFile.getAbsolutePath()); + cmd.setConf(getDecryptionConfiguration()); - ParquetReadOptions readOptions = - ParquetReadOptions.builder().withDecryption(decryptionProps).build(); - InputFile inputFile = HadoopInputFile.fromPath(new Path(footerEncryptedFile.getPath()), verifyConf); + int rc = cmd.run(); + assertEquals(0, rc); - try { - ParquetFileReader.open(inputFile, readOptions); - fail("Expected ParquetCryptoRuntimeException for footer encrypted file without keys"); - } catch (ParquetCryptoRuntimeException e) { - } + boolean foundColumnSummary = false; + boolean foundSsn = false; - ByteArrayOutputStream out = new ByteArrayOutputStream(); - enableCliDecryptionKeys(); - Logger logger = createLogger(new PrintStream(out)); - EncryptionInfoCommand cmd = new EncryptionInfoCommand(logger); - cmd.targets = java.util.Arrays.asList(footerEncryptedFile.getAbsolutePath()); - cmd.setConf(new Configuration()); + for (org.slf4j.event.LoggingEvent event : events) { + String message = event.getMessage(); + if (message.contains("Column encryption summary:")) { + foundColumnSummary = true; + } + if (message.contains("ssn")) { + foundSsn = true; + } + } - int rc = cmd.run(); - assertEquals(0, rc); - String output = out.toString(); - assertTrue(output.contains("Encryption type: ENCRYPTED_FOOTER")); - assertTrue(output.contains("Footer is encrypted")); + assertTrue("Should contain column encryption summary", foundColumnSummary); + assertTrue("Should contain ssn column", foundSsn); + }); } }