You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by bl...@apache.org on 2015/03/25 00:06:39 UTC

incubator-parquet-mr git commit: PARQUET-204: add parquet-schema directory support

Repository: incubator-parquet-mr
Updated Branches:
  refs/heads/master cb7f6a8cb -> fd3085ed3


PARQUET-204: add parquet-schema directory support

Author: Neville Li <ne...@spotify.com>

Closes #136 from nevillelyh/neville/PARQUET-204 and squashes the following commits:

633829b [Neville Li] PARQUET-204: add parquet-schema directory support
7aa8581 [Neville Li] PARQUET-203: consolidate PathFilter for hidden files


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

Branch: refs/heads/master
Commit: fd3085ed31d920e8ca6bba391e75d1423ed8b607
Parents: cb7f6a8
Author: Neville Li <ne...@spotify.com>
Authored: Tue Mar 24 16:06:26 2015 -0700
Committer: Ryan Blue <bl...@apache.org>
Committed: Tue Mar 24 16:06:26 2015 -0700

----------------------------------------------------------------------
 .../java/parquet/hadoop/ParquetFileReader.java  |  9 ++----
 .../java/parquet/hadoop/ParquetInputFormat.java | 10 ++----
 .../main/java/parquet/hadoop/ParquetReader.java |  9 ++----
 .../main/java/parquet/hadoop/PrintFooter.java   |  9 ++----
 .../parquet/hadoop/util/HiddenFileFilter.java   | 33 ++++++++++++++++++++
 .../parquet/hadoop/TestParquetFileWriter.java   |  9 ++----
 .../tools/command/ShowSchemaCommand.java        | 19 ++++++++++-
 7 files changed, 61 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/fd3085ed/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
index de94b12..99f52c8 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 
 import parquet.Log;
 import parquet.bytes.BytesInput;
@@ -75,6 +74,7 @@ import parquet.hadoop.ColumnChunkPageReadStore.ColumnChunkPageReader;
 import parquet.hadoop.metadata.BlockMetaData;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.hadoop.util.HiddenFileFilter;
 import parquet.hadoop.util.counters.BenchmarkCounter;
 import parquet.io.ParquetDecodingException;
 
@@ -299,12 +299,7 @@ public class ParquetFileReader implements Closeable {
   private static List<FileStatus> listFiles(Configuration conf, FileStatus fileStatus) throws IOException {
     if (fileStatus.isDir()) {
       FileSystem fs = fileStatus.getPath().getFileSystem(conf);
-      FileStatus[] list = fs.listStatus(fileStatus.getPath(), new PathFilter() {
-        @Override
-        public boolean accept(Path p) {
-          return !p.getName().startsWith("_") && !p.getName().startsWith(".");
-        }
-      });
+      FileStatus[] list = fs.listStatus(fileStatus.getPath(), HiddenFileFilter.INSTANCE);
       List<FileStatus> result = new ArrayList<FileStatus>();
       for (FileStatus sub : list) {
         result.addAll(listFiles(conf, sub));

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/fd3085ed/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
index 8728965..6b7b9b0 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
@@ -65,6 +65,7 @@ import parquet.hadoop.metadata.GlobalMetaData;
 import parquet.hadoop.metadata.ParquetMetadata;
 import parquet.hadoop.util.ConfigurationUtil;
 import parquet.hadoop.util.ContextUtil;
+import parquet.hadoop.util.HiddenFileFilter;
 import parquet.hadoop.util.SerializationUtil;
 import parquet.io.ParquetDecodingException;
 import parquet.schema.MessageType;
@@ -346,7 +347,7 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void, T> {
       if (file.isDir()) {
         Path p = file.getPath();
         FileSystem fs = p.getFileSystem(conf);
-        staticAddInputPathRecursively(result, fs, p, hiddenFileFilter);
+        staticAddInputPathRecursively(result, fs, p, HiddenFileFilter.INSTANCE);
       } else {
         result.add(file);
       }
@@ -367,13 +368,6 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void, T> {
     }
   }
 
-  private static final PathFilter hiddenFileFilter = new PathFilter(){
-    public boolean accept(Path p){
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
-
   /**
    * @param jobContext the current job context
    * @return the footers for the files

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/fd3085ed/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java
index 4231ae4..7c3ecbb 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java
@@ -31,13 +31,13 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import org.apache.hadoop.fs.PathFilter;
 import parquet.filter.UnboundRecordFilter;
 import parquet.filter2.compat.FilterCompat;
 import parquet.filter2.compat.FilterCompat.Filter;
 import parquet.filter2.compat.RowGroupFilter;
 import parquet.hadoop.api.ReadSupport;
 import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.util.HiddenFileFilter;
 import parquet.schema.MessageType;
 
 /**
@@ -110,12 +110,7 @@ public class ParquetReader<T> implements Closeable {
     this.conf = conf;
 
     FileSystem fs = file.getFileSystem(conf);
-    List<FileStatus> statuses = Arrays.asList(fs.listStatus(file, new PathFilter() {
-      @Override
-      public boolean accept(Path p) {
-        return !p.getName().startsWith("_") && !p.getName().startsWith(".");
-      }
-    }));
+    List<FileStatus> statuses = Arrays.asList(fs.listStatus(file, HiddenFileFilter.INSTANCE));
     List<Footer> footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(conf, statuses, false);
     this.footersIterator = footers.iterator();
   }

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/fd3085ed/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java b/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java
index 5f1d271..0728f8f 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 
 import parquet.column.ColumnDescriptor;
 import parquet.column.Encoding;
@@ -50,6 +49,7 @@ import parquet.column.statistics.Statistics;
 import parquet.hadoop.metadata.BlockMetaData;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.hadoop.util.HiddenFileFilter;
 import parquet.io.ParquetDecodingException;
 import parquet.schema.MessageType;
 
@@ -82,12 +82,7 @@ public class PrintFooter {
       List<FileStatus> statuses;
       if (fileStatus.isDir()) {
         System.out.println("listing files in " + fileStatus.getPath());
-        statuses = Arrays.asList(fs.listStatus(fileStatus.getPath(), new PathFilter() {
-          @Override
-          public boolean accept(Path path) {
-            return !path.getName().startsWith("_");
-          }
-        }));
+        statuses = Arrays.asList(fs.listStatus(fileStatus.getPath(), HiddenFileFilter.INSTANCE));
       } else {
         statuses = new ArrayList<FileStatus>();
         statuses.add(fileStatus);

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/fd3085ed/parquet-hadoop/src/main/java/parquet/hadoop/util/HiddenFileFilter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/util/HiddenFileFilter.java b/parquet-hadoop/src/main/java/parquet/hadoop/util/HiddenFileFilter.java
new file mode 100644
index 0000000..0bc2de6
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/util/HiddenFileFilter.java
@@ -0,0 +1,33 @@
+/*
+ * 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 parquet.hadoop.util;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+
+public class HiddenFileFilter implements PathFilter {
+  public static final HiddenFileFilter INSTANCE = new HiddenFileFilter();
+
+  private HiddenFileFilter() {}
+
+  @Override
+  public boolean accept(Path p) {
+    return !p.getName().startsWith("_") && !p.getName().startsWith(".");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/fd3085ed/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java b/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
index 4a81b85..e991bd2 100644
--- a/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.junit.Test;
 import parquet.Log;
 import parquet.bytes.BytesInput;
@@ -37,6 +36,7 @@ import parquet.column.statistics.BinaryStatistics;
 import parquet.column.statistics.LongStatistics;
 import parquet.format.Statistics;
 import parquet.hadoop.metadata.*;
+import parquet.hadoop.util.HiddenFileFilter;
 import parquet.io.api.Binary;
 import parquet.schema.MessageType;
 import parquet.schema.MessageTypeParser;
@@ -325,12 +325,7 @@ public class TestParquetFileWriter {
 
     validateFooters(metadata);
 
-    footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(configuration, Arrays.asList(fs.listStatus(testDirPath, new PathFilter() {
-      @Override
-      public boolean accept(Path p) {
-        return !p.getName().startsWith("_");
-      }
-    })), false);
+    footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(configuration, Arrays.asList(fs.listStatus(testDirPath, HiddenFileFilter.INSTANCE)), false);
     validateFooters(footers);
 
     fs.delete(metadataFile.getPath(), false);

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/fd3085ed/parquet-tools/src/main/java/parquet/tools/command/ShowSchemaCommand.java
----------------------------------------------------------------------
diff --git a/parquet-tools/src/main/java/parquet/tools/command/ShowSchemaCommand.java b/parquet-tools/src/main/java/parquet/tools/command/ShowSchemaCommand.java
index a933aea..11c4304 100644
--- a/parquet-tools/src/main/java/parquet/tools/command/ShowSchemaCommand.java
+++ b/parquet-tools/src/main/java/parquet/tools/command/ShowSchemaCommand.java
@@ -25,10 +25,13 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 import parquet.hadoop.ParquetFileReader;
 import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.hadoop.util.HiddenFileFilter;
 import parquet.schema.MessageType;
 import parquet.tools.Main;
 import parquet.tools.util.MetadataUtils;
@@ -71,7 +74,21 @@ public class ShowSchemaCommand extends ArgsOnlyCommand {
     String input = args[0];
 
     Configuration conf = new Configuration();
-    ParquetMetadata metaData = ParquetFileReader.readFooter(conf, new Path(input));
+    ParquetMetadata metaData;
+
+    Path path = new Path(input);
+    FileSystem fs = path.getFileSystem(conf);
+    Path file;
+    if (fs.isDirectory(path)) {
+      FileStatus[] statuses = fs.listStatus(path, HiddenFileFilter.INSTANCE);
+      if (statuses.length == 0) {
+        throw new RuntimeException("Directory " + path.toString() + " is empty");
+      }
+      file = statuses[0].getPath();
+    } else {
+      file = path;
+    }
+    metaData = ParquetFileReader.readFooter(conf, file);
     MessageType schema = metaData.getFileMetaData().getSchema();
 
     Main.out.println(schema);