diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowFooterCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowFooterCommand.java index 49a67cb723..06d71a55a2 100644 --- a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowFooterCommand.java +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowFooterCommand.java @@ -24,6 +24,7 @@ import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.PropertyAccessor; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -64,6 +65,11 @@ public int run() throws IOException { return 0; } + abstract class MixIn { + @JsonIgnore + abstract int getInputFile(); + } + private String readFooter(InputFile inputFile) throws JsonProcessingException, IOException { String json; try (ParquetFileReader reader = ParquetFileReader.open(inputFile)) { @@ -71,6 +77,7 @@ private String readFooter(InputFile inputFile) throws JsonProcessingException, I ObjectMapper mapper = RawUtils.createObjectMapper(); mapper.setVisibility(PropertyAccessor.ALL, Visibility.NONE); mapper.setVisibility(PropertyAccessor.FIELD, Visibility.ANY); + mapper.addMixIn(ParquetMetadata.class, MixIn.class); json = mapper.writerWithDefaultPrettyPrinter().writeValueAsString(footer); } return json; diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 64f0f514b7..87586047ee 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -95,6 +95,11 @@ jackson-databind ${jackson-databind.version} + + ${jackson.groupId} + jackson-annotations + ${jackson-databind.version} + org.xerial.snappy snappy-java diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index ce154e0aa1..ee1b5fc06e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -613,8 +613,10 @@ private static final ParquetMetadata readFooter( // Regular file, or encrypted file with plaintext footer if (!encryptedFooterMode) { - return converter.readParquetMetadata( + ParquetMetadata parquetMetadata = converter.readParquetMetadata( footerBytesStream, options.getMetadataFilter(), fileDecryptor, false, fileMetadataLength); + parquetMetadata.setInputFile(file); + return parquetMetadata; } // Encrypted file with encrypted footer @@ -625,7 +627,10 @@ private static final ParquetMetadata readFooter( fileDecryptor.setFileCryptoMetaData( fileCryptoMetaData.getEncryption_algorithm(), true, fileCryptoMetaData.getKey_metadata()); // footer length is required only for signed plaintext footers - return converter.readParquetMetadata(footerBytesStream, options.getMetadataFilter(), fileDecryptor, true, 0); + ParquetMetadata parquetMetadata = + converter.readParquetMetadata(footerBytesStream, options.getMetadataFilter(), fileDecryptor, true, 0); + parquetMetadata.setInputFile(file); + return parquetMetadata; } /** @@ -824,12 +829,19 @@ public ParquetFileReader(Configuration conf, Path file, ParquetMetadata footer) } public ParquetFileReader(InputFile file, ParquetReadOptions options) throws IOException { + this(file, options, null); + } + + public ParquetFileReader(InputFile file, ParquetReadOptions options, ParquetMetadata footer) throws IOException { this.converter = new ParquetMetadataConverter(options); this.file = file; this.f = file.newStream(); this.options = options; try { - this.footer = readFooter(file, options, f, converter); + if (footer == null) { + footer = readFooter(file, options, f, converter); + } + this.footer = footer; } catch (Exception e) { // In case that reading footer throws an exception in the constructor, the new stream // should be closed. Otherwise, there's no way to close this outside. diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputSplit.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputSplit.java index e851f3ed9b..8b8cdadd69 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputSplit.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputSplit.java @@ -18,6 +18,7 @@ */ package org.apache.parquet.hadoop; +import com.fasterxml.jackson.annotation.JsonIgnore; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInput; @@ -36,6 +37,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.MessageTypeParser; @@ -55,6 +57,9 @@ public class ParquetInputSplit extends FileSplit implements Writable { private long end; private long[] rowGroupOffsets; + @JsonIgnore + private volatile ParquetMetadata footer; + /** * Writables must have a parameterless constructor */ @@ -222,6 +227,14 @@ public long[] getRowGroupOffsets() { return rowGroupOffsets; } + public ParquetMetadata getFooter() { + return footer; + } + + public void setFooter(ParquetMetadata footer) { + this.footer = footer; + } + @Override public String toString() { String hosts; diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java index b217116aac..a809159267 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordReader.java @@ -46,6 +46,7 @@ import org.apache.parquet.hadoop.util.ContextUtil; import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.hadoop.util.counters.BenchmarkCounter; +import org.apache.parquet.io.InputFile; import org.apache.parquet.io.ParquetDecodingException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -155,8 +156,13 @@ private void initializeInternalReader(ParquetInputSplit split, Configuration con } // open a reader with the metadata filter - ParquetFileReader reader = - ParquetFileReader.open(HadoopInputFile.fromPath(path, configuration), optionsBuilder.build()); + InputFile inputFile; + if (split.getFooter() != null && split.getFooter().getInputFile() != null) { + inputFile = split.getFooter().getInputFile(); + } else { + inputFile = HadoopInputFile.fromPath(path, configuration); + } + ParquetFileReader reader = new ParquetFileReader(inputFile, optionsBuilder.build(), split.getFooter()); if (rowGroupOffsets != null) { // verify a row group was found for each offset diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java index e30e872a6a..2819067c5f 100755 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java @@ -18,11 +18,13 @@ */ package org.apache.parquet.hadoop.metadata; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.StringReader; import java.io.StringWriter; import java.util.List; +import org.apache.parquet.io.InputFile; /** * Meta Data block stored in the footer of the file @@ -84,6 +86,9 @@ public static ParquetMetadata fromJSON(String json) { private final FileMetaData fileMetaData; private final List blocks; + @JsonIgnore + private volatile InputFile inputFile; + /** * @param fileMetaData file level metadata * @param blocks block level metadata @@ -107,6 +112,22 @@ public FileMetaData getFileMetaData() { return fileMetaData; } + /** + * Reuse the inputFile in ParquetFileReader if it is not null + * @return + */ + public InputFile getInputFile() { + return inputFile; + } + + /** + * + * @param inputFile Cache the inputFile in readFooter method and reuse it in ParquetFileReader + */ + public void setInputFile(InputFile inputFile) { + this.inputFile = inputFile; + } + @Override public String toString() { return "ParquetMetaData{" + fileMetaData + ", blocks: " + blocks + "}";