You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ju...@apache.org on 2016/08/03 21:22:57 UTC

parquet-mr git commit: PARQUET-669: allow reading footers from provided file listing and streams

Repository: parquet-mr
Updated Branches:
  refs/heads/master ea402becc -> 76a2ac814


PARQUET-669: allow reading footers from provided file listing and streams

The use case is that I want to reuse existing listing of files and avoid doing it again when opening streams. This is in case where filesystem.open is expensive but you have other means of obtaining input stream for a file.

Author: Robert Kruszewski <ro...@palantir.com>

Closes #357 from robert3005/robertk/allow-reading-footers-from-streams and squashes the following commits:

4d8a54c [Robert Kruszewski] allow reading footers from provided file listing and streams


Project: http://git-wip-us.apache.org/repos/asf/parquet-mr/repo
Commit: http://git-wip-us.apache.org/repos/asf/parquet-mr/commit/76a2ac81
Tree: http://git-wip-us.apache.org/repos/asf/parquet-mr/tree/76a2ac81
Diff: http://git-wip-us.apache.org/repos/asf/parquet-mr/diff/76a2ac81

Branch: refs/heads/master
Commit: 76a2ac814caa194c46be1cf7a3f5dc129546b1c1
Parents: ea402be
Author: Robert Kruszewski <ro...@palantir.com>
Authored: Wed Aug 3 14:22:27 2016 -0700
Committer: Julien Le Dem <ju...@dremio.com>
Committed: Wed Aug 3 14:22:27 2016 -0700

----------------------------------------------------------------------
 .../parquet/hadoop/ParquetFileReader.java       | 28 +++++++++++++-------
 1 file changed, 18 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/76a2ac81/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.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 a761f2e..83542d5 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
@@ -434,22 +434,30 @@ public class ParquetFileReader implements Closeable {
     FileSystem fileSystem = file.getPath().getFileSystem(configuration);
     FSDataInputStream in = fileSystem.open(file.getPath());
     try {
-      return readFooter(file, in, filter);
+      return readFooter(file.getLen(), file.getPath().toString(), in, filter);
     } finally {
       in.close();
     }
   }
 
-  private static final ParquetMetadata readFooter(FileStatus file, FSDataInputStream f, MetadataFilter filter) throws IOException {
-    long l = file.getLen();
+  /**
+   * Reads the meta data block in the footer of the file using provided input stream
+   * @param fileLen length of the file
+   * @param filePath file location
+   * @param f input stream for the file
+   * @param filter the filter to apply to row groups
+   * @return the metadata blocks in the footer
+   * @throws IOException if an error occurs while reading the file
+   */
+  public static final ParquetMetadata readFooter(long fileLen, String filePath, FSDataInputStream f, MetadataFilter filter) throws IOException {
     if (Log.DEBUG) {
-      LOG.debug("File length " + l);
+      LOG.debug("File length " + fileLen);
     }
     int FOOTER_LENGTH_SIZE = 4;
-    if (l < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + footerIndex + MAGIC
-      throw new RuntimeException(file.getPath() + " is not a Parquet file (too small)");
+    if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + footerIndex + MAGIC
+      throw new RuntimeException(filePath + " is not a Parquet file (too small)");
     }
-    long footerLengthIndex = l - FOOTER_LENGTH_SIZE - MAGIC.length;
+    long footerLengthIndex = fileLen - FOOTER_LENGTH_SIZE - MAGIC.length;
     if (Log.DEBUG) {
       LOG.debug("reading footer index at " + footerLengthIndex);
     }
@@ -459,7 +467,7 @@ public class ParquetFileReader implements Closeable {
     byte[] magic = new byte[MAGIC.length];
     f.readFully(magic);
     if (!Arrays.equals(MAGIC, magic)) {
-      throw new RuntimeException(file.getPath() + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) + " but found " + Arrays.toString(magic));
+      throw new RuntimeException(filePath + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) + " but found " + Arrays.toString(magic));
     }
     long footerIndex = footerLengthIndex - footerLength;
     if (Log.DEBUG) {
@@ -555,7 +563,7 @@ public class ParquetFileReader implements Closeable {
     FileSystem fs = file.getFileSystem(conf);
     this.fileStatus = fs.getFileStatus(file);
     this.f = fs.open(file);
-    this.footer = readFooter(fileStatus, f, filter);
+    this.footer = readFooter(fileStatus.getLen(), fileStatus.getPath().toString(), f, filter);
     this.fileMetaData = footer.getFileMetaData();
     this.blocks = footer.getBlocks();
     for (ColumnDescriptor col : footer.getFileMetaData().getSchema().getColumns()) {
@@ -594,7 +602,7 @@ public class ParquetFileReader implements Closeable {
     if (footer == null) {
       try {
         // don't read the row groups because this.blocks is always set
-        this.footer = readFooter(fileStatus, f, SKIP_ROW_GROUPS);
+        this.footer = readFooter(fileStatus.getLen(), fileStatus.getPath().toString(), f, SKIP_ROW_GROUPS);
       } catch (IOException e) {
         throw new ParquetDecodingException("Unable to read file footer", e);
       }