org.apache.avro
avro
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/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
new file mode 100644
index 0000000000..d15446c154
--- /dev/null
+++ b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/TestEncryptionInfoCommand.java
@@ -0,0 +1,299 @@
+/*
+ * 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.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+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.DecryptionPropertiesFactory;
+import org.apache.parquet.crypto.EncryptionPropertiesFactory;
+import org.apache.parquet.crypto.FileDecryptionProperties;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+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.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.LoggerFactory;
+
+/**
+ * This test checks the CLI functionality for encryption info.
+ */
+public class TestEncryptionInfoCommand extends ParquetFileTest {
+
+ 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();
+ createTestFiles();
+ }
+
+ private void createTestFiles() throws IOException {
+ 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 File createParquetFile(String filename, EncMode mode) throws IOException {
+ File file = new File(getTempFolder(), filename);
+ Path fsPath = new Path(file.getPath());
+
+ Configuration conf = getHadoopConfiguration(mode);
+ FileEncryptionProperties fileEncryptionProperties = null;
+
+ try {
+ if (null == conf) {
+ conf = new Configuration();
+ } else {
+ EncryptionPropertiesFactory cryptoFactory = EncryptionPropertiesFactory.loadFactory(conf);
+ fileEncryptionProperties = cryptoFactory.getFileEncryptionProperties(conf, fsPath, null);
+ }
+ } catch (Exception e) {
+ throw new IOException("Failed to create encryption properties for " + filename, e);
+ }
+
+ MessageType schema = Types.buildMessage()
+ .required(PrimitiveType.PrimitiveTypeName.INT32)
+ .named("id")
+ .required(PrimitiveType.PrimitiveTypeName.BINARY)
+ .named("ssn")
+ .required(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(ParquetFileWriter.Mode.OVERWRITE)
+ .withCompressionCodec(CompressionCodecName.UNCOMPRESSED)
+ .withRowGroupSize(1024)
+ .withPageSize(1024)
+ .withDictionaryPageSize(512)
+ .withDictionaryEncoding(true)
+ .withValidation(false)
+ .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_2_0)
+ .withEncryption(fileEncryptionProperties);
+
+ 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 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 + ":" + 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 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 = 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 = Arrays.asList(columnEncryptedFile.getAbsolutePath());
+ cmd.setConf(new Configuration());
+
+ try {
+ cmd.run();
+ fail("Expected ParquetCryptoRuntimeException for encrypted file without keys");
+ } 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 = Arrays.asList(footerEncryptedFile.getAbsolutePath());
+ cmd.setConf(getDecryptionConfiguration());
+
+ int result = cmd.run();
+ assertEquals(0, result);
+ }
+
+ @Test
+ public void testCommandWorksWithRealFile() throws IOException {
+ File file = parquetFile();
+ EncryptionInfoCommand command = new EncryptionInfoCommand(createLogger());
+ command.targets = Arrays.asList(file.getAbsolutePath());
+ command.setConf(new Configuration());
+
+ int result = command.run();
+ assertEquals(0, result);
+ }
+
+ @Test
+ public void testCommandExists() {
+ new EncryptionInfoCommand(LoggerFactory.getLogger(TestEncryptionInfoCommand.class));
+ }
+
+ @Test
+ public void testColumnEncryptedFileWithKeys() throws IOException {
+ assertTrue(columnEncryptedFile.exists());
+
+ EncryptionInfoCommand cmd = new EncryptionInfoCommand(createLogger());
+ cmd.targets = Arrays.asList(columnEncryptedFile.getAbsolutePath());
+ cmd.setConf(getDecryptionConfiguration());
+ int rc = cmd.run();
+ assertEquals(0, rc);
+
+ 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,
+ ParquetReadOptions.builder()
+ .withDecryption(fileDecryptionProperties)
+ .build())) {
+ footer = reader.getFooter();
+ }
+ Set encrypted = EncryptionInfoCommand.findEncryptedColumns(footer);
+ assertEquals(Collections.singleton("ssn"), encrypted);
+ }
+
+ @Test
+ public void testFooterEncryptedFileWithKeys() throws IOException {
+ withLogger((logger, events) -> {
+ EncryptionInfoCommand cmd = new EncryptionInfoCommand(logger);
+ cmd.targets = Arrays.asList(footerEncryptedFile.getAbsolutePath());
+ cmd.setConf(getDecryptionConfiguration());
+
+ int rc = cmd.run();
+ assertEquals(0, rc);
+
+ boolean foundColumnSummary = false;
+ boolean foundSsn = false;
+
+ 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;
+ }
+ }
+
+ assertTrue("Should contain column encryption summary", foundColumnSummary);
+ assertTrue("Should contain ssn column", foundSsn);
+ });
+ }
+}