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
8 changes: 8 additions & 0 deletions parquet-cli/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,14 @@
<artifactId>parquet-hadoop</artifactId>
<version>${project.version}</version>
</dependency>
<!-- Bring in helper test classes like InMemoryKMS used by CLI encryption tests -->
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
<classifier>tests</classifier>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
Expand Down
1 change: 1 addition & 0 deletions parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
@@ -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.
* <p>
* 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 = "<parquet path>", required = true)
List<String> 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();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it would be nice to also print out details about the encryption algorithm, wouldn't it?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah that's a great point will add support

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<String> 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<ColumnChunkMetaData, String>() {
@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<String> strings) {
int size = 0;
for (String s : strings) {
size = Math.max(size, s.length());
}
return size;
}

/** Visible for test */
static Set<String> findEncryptedColumns(ParquetMetadata footer) {
Set<String> 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<String> encryptedColumns = new TreeSet<>();
Set<String> 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<String> encryptedColumns;
private final Set<String> unencryptedColumns;

public ColumnEncryptionInfo(Set<String> encryptedColumns, Set<String> unencryptedColumns) {
this.encryptedColumns = encryptedColumns;
this.unencryptedColumns = unencryptedColumns;
}

public Set<String> getEncryptedColumns() {
return encryptedColumns;
}

public Set<String> getUnencryptedColumns() {
return unencryptedColumns;
}

public int getTotalColumns() {
return encryptedColumns.size() + unencryptedColumns.size();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,10 @@ protected static Logger createLogger() {
return console;
}

protected static Logger createLogger(java.io.PrintStream out) {
return createLogger();
}

@FunctionalInterface
public interface ThrowableBiConsumer<T, U> {
void accept(T t, U u) throws Exception;
Expand Down
Loading