parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From b...@apache.org
Subject incubator-parquet-mr git commit: PARQUET-124: normalize path checking to prevent mismatch between URI and ...
Date Fri, 30 Jan 2015 01:29:14 GMT
Repository: incubator-parquet-mr
Updated Branches:
  refs/heads/master d7dd228d0 -> e505e1fea


PARQUET-124: normalize path checking to prevent mismatch between URI and ...

...path

Author: Chris Albright <calbright@cj.com>

Closes #79 from chrisalbright/master and squashes the following commits:

b1b0086 [Chris Albright] Merge remote-tracking branch 'upstream/master'
9669427 [Chris Albright] PARQUET-124: Adding test (Thanks Ryan Blue) that proves mergeFooters
was failing
8e342ed [Chris Albright] PARQUET-124: normalize path checking to prevent mismatch between
URI and path


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/e505e1fe
Tree: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/tree/e505e1fe
Diff: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/diff/e505e1fe

Branch: refs/heads/master
Commit: e505e1fea57e0ab9f1d5edab92546d778a5f41e0
Parents: d7dd228
Author: Chris Albright <calbright@cj.com>
Authored: Thu Jan 29 17:29:06 2015 -0800
Committer: Ryan Blue <blue@apache.org>
Committed: Thu Jan 29 17:29:06 2015 -0800

----------------------------------------------------------------------
 .../java/parquet/hadoop/ParquetFileWriter.java  | 18 ++---
 .../parquet/hadoop/TestParquetFileWriter.java   | 80 ++++++++++++++------
 2 files changed, 65 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/e505e1fe/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
index 42d91a4..1c651af 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
@@ -412,22 +412,22 @@ public class ParquetFileWriter {
     metadata.close();
   }
 
-  private static ParquetMetadata mergeFooters(Path root, List<Footer> footers) {
-    String rootPath = root.toString();
+  static ParquetMetadata mergeFooters(Path root, List<Footer> footers) {
+    String rootPath = root.toUri().getPath();
     GlobalMetaData fileMetaData = null;
     List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
     for (Footer footer : footers) {
-      String path = footer.getFile().toString();
-      if (!path.startsWith(rootPath)) {
-        throw new ParquetEncodingException(path + " invalid: all the files must be contained
in the root " + root);
+        String footerPath = footer.getFile().toUri().getPath();
+      if (!footerPath.startsWith(rootPath)) {
+        throw new ParquetEncodingException(footerPath + " invalid: all the files must be
contained in the root " + root);
       }
-      path = path.substring(rootPath.length());
-      while (path.startsWith("/")) {
-        path = path.substring(1);
+      footerPath = footerPath.substring(rootPath.length());
+      while (footerPath.startsWith("/")) {
+        footerPath = footerPath.substring(1);
       }
       fileMetaData = mergeInto(footer.getParquetMetadata().getFileMetaData(), fileMetaData);
       for (BlockMetaData block : footer.getParquetMetadata().getBlocks()) {
-        block.setPath(path);
+        block.setPath(footerPath);
         blocks.add(block);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/e505e1fe/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 5d0b17f..2775303 100644
--- a/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
@@ -15,21 +15,6 @@
  */
 package parquet.hadoop;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static parquet.column.Encoding.BIT_PACKED;
-import static parquet.column.Encoding.PLAIN;
-import static parquet.hadoop.TestUtils.enforceEmptyDir;
-import static parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
-import static parquet.schema.Type.Repetition.OPTIONAL;
-import static parquet.schema.Type.Repetition.REPEATED;
-import static parquet.schema.Type.Repetition.REQUIRED;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -37,7 +22,6 @@ 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;
 import parquet.column.ColumnDescriptor;
@@ -48,19 +32,24 @@ import parquet.column.page.PageReadStore;
 import parquet.column.page.PageReader;
 import parquet.column.statistics.BinaryStatistics;
 import parquet.column.statistics.LongStatistics;
-import parquet.hadoop.metadata.BlockMetaData;
-import parquet.hadoop.metadata.ColumnChunkMetaData;
-import parquet.hadoop.metadata.CompressionCodecName;
-import parquet.hadoop.metadata.FileMetaData;
-import parquet.hadoop.metadata.GlobalMetaData;
-import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.format.Statistics;
+import parquet.hadoop.metadata.*;
 import parquet.io.api.Binary;
 import parquet.schema.MessageType;
 import parquet.schema.MessageTypeParser;
 import parquet.schema.PrimitiveType;
 import parquet.schema.PrimitiveType.PrimitiveTypeName;
-import parquet.format.Statistics;
-import parquet.format.converter.ParquetMetadataConverter;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
+
+import static org.junit.Assert.*;
+import static parquet.column.Encoding.BIT_PACKED;
+import static parquet.column.Encoding.PLAIN;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static parquet.schema.Type.Repetition.*;
+import static parquet.hadoop.TestUtils.enforceEmptyDir;
 
 import parquet.example.data.Group;
 import parquet.example.data.simple.SimpleGroup;
@@ -433,4 +422,47 @@ public class TestParquetFileWriter {
 
   }
 
+  @Test
+  public void testMergeFooters() {
+    List<BlockMetaData> oneBlocks = new ArrayList<BlockMetaData>();
+    oneBlocks.add(new BlockMetaData());
+    oneBlocks.add(new BlockMetaData());
+    List<BlockMetaData> twoBlocks = new ArrayList<BlockMetaData>();
+    twoBlocks.add(new BlockMetaData());
+    List<BlockMetaData> expected = new ArrayList<BlockMetaData>();
+    expected.addAll(oneBlocks);
+    expected.addAll(twoBlocks);
+
+    Footer one = new Footer(new Path("file:/tmp/output/one.parquet"),
+        new ParquetMetadata(new FileMetaData(
+            new MessageType("root1",
+                new PrimitiveType(REPEATED, BINARY, "a"),
+                new PrimitiveType(OPTIONAL, BINARY, "b")),
+            new HashMap<String, String>(), "test"),
+        oneBlocks));
+
+    Footer two = new Footer(new Path("/tmp/output/two.parquet"),
+        new ParquetMetadata(new FileMetaData(
+            new MessageType("root2",
+                new PrimitiveType(REQUIRED, BINARY, "c")),
+            new HashMap<String, String>(), "test2"),
+            twoBlocks));
+
+    List<Footer> footers = new ArrayList<Footer>();
+    footers.add(one);
+    footers.add(two);
+
+    ParquetMetadata merged = ParquetFileWriter.mergeFooters(
+        new Path("/tmp"), footers);
+
+    assertEquals(
+        new MessageType("root1",
+            new PrimitiveType(REPEATED, BINARY, "a"),
+            new PrimitiveType(OPTIONAL, BINARY, "b"),
+            new PrimitiveType(REQUIRED, BINARY, "c")),
+        merged.getFileMetaData().getSchema());
+
+    assertEquals("Should have all blocks", expected, merged.getBlocks());
+  }
+
 }


Mime
View raw message