parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From b...@apache.org
Subject parquet-mr git commit: PARQUET-363: Allow empty schema groups.
Date Fri, 11 Sep 2015 22:14:09 GMT
Repository: parquet-mr
Updated Branches:
  refs/heads/master 9962a0fd0 -> f203d809d


PARQUET-363: Allow empty schema groups.

This removes the check added in PARQUET-278 that rejects schema groups
that have no fields. Selecting 0 columns from a file is allowed and used
by Hive and SparkSQL to implement queries like `select count(1) ...`

Author: Ryan Blue <blue@apache.org>

Closes #263 from rdblue/PARQUET-363-allow-empty-groups and squashes the following commits:

ab370f1 [Ryan Blue] PARQUET-363: Update Type builder tests to allow empty groups.
926932b [Ryan Blue] PARQUET-363: Add write-side schema validation.
365f30d [Ryan Blue] PARQUET-363: Allow empty schema groups.


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

Branch: refs/heads/master
Commit: f203d809d7b94501a2e5409c667ba86206480f90
Parents: 9962a0f
Author: Ryan Blue <blue@apache.org>
Authored: Fri Sep 11 15:14:00 2015 -0700
Committer: Ryan Blue <blue@apache.org>
Committed: Fri Sep 11 15:14:00 2015 -0700

----------------------------------------------------------------------
 .../org/apache/parquet/schema/GroupType.java    |  3 -
 .../org/apache/parquet/schema/TypeUtil.java     | 45 ++++++++++
 .../java/org/apache/parquet/schema/Types.java   |  2 -
 .../apache/parquet/schema/TestMessageType.java  | 13 ---
 .../apache/parquet/schema/TestTypeBuilders.java | 40 +++------
 .../org/apache/parquet/schema/TestTypeUtil.java | 89 +++++++++++++++++++
 .../parquet/hadoop/ParquetFileWriter.java       |  2 +
 .../hadoop/example/ExampleParquetWriter.java    | 93 ++++++++++++++++++++
 .../hadoop/example/GroupWriteSupport.java       | 14 ++-
 .../parquet/hadoop/TestParquetWriter.java       | 37 ++++++++
 .../org/apache/parquet/hadoop/TestUtils.java    | 19 ++++
 11 files changed, 307 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java b/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
index bf98bfd..027fbc0 100644
--- a/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
@@ -88,9 +88,6 @@ public class GroupType extends Type {
    */
   GroupType(Repetition repetition, String name, OriginalType originalType, List<Type>
fields, ID id) {
     super(name, repetition, originalType, id);
-    if (fields.isEmpty()) {
-      throw new InvalidSchemaException("A group type can not be empty. Parquet does not support
empty group without leaves. Empty group: " + name);
-    }
     this.fields = fields;
     this.indexByName = new HashMap<String, Integer>();
     for (int i = 0; i < fields.size(); i++) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-column/src/main/java/org/apache/parquet/schema/TypeUtil.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/TypeUtil.java b/parquet-column/src/main/java/org/apache/parquet/schema/TypeUtil.java
new file mode 100644
index 0000000..2a9d442
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/TypeUtil.java
@@ -0,0 +1,45 @@
+/**
+ * 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.schema;
+
+public class TypeUtil {
+  public static void checkValidWriteSchema(GroupType schema) {
+    schema.accept(new TypeVisitor() {
+      @Override
+      public void visit(GroupType groupType) {
+        if (groupType.getFieldCount() <= 0) {
+          throw new InvalidSchemaException(
+              "Cannot write a schema with an empty group: " + groupType);
+        }
+        for (Type type : groupType.getFields()) {
+          type.accept(this);
+        }
+      }
+
+      @Override
+      public void visit(MessageType messageType) {
+        visit((GroupType) messageType);
+      }
+
+      @Override
+      public void visit(PrimitiveType primitiveType) {
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/Types.java b/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
index 65e1ef0..947a845 100644
--- a/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
@@ -619,8 +619,6 @@ public class Types {
 
     @Override
     protected GroupType build(String name) {
-      Preconditions.checkState(!fields.isEmpty(),
-          "Cannot build an empty group");
       return new GroupType(repetition, name, originalType, fields, id);
     }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java b/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
index b730ec8..ca5d939 100644
--- a/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
+++ b/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
@@ -146,17 +146,4 @@ public class TestMessageType {
     assertEquals(schema.toString(), schema2.toString());
   }
 
-  @Test
-  public void testEmptyGroup() {
-    try {
-      MessageType t5 = new MessageType("root1",
-          new GroupType(REQUIRED, "g1"),
-          new GroupType(REQUIRED, "g2",
-              new PrimitiveType(OPTIONAL, BINARY, "b")));
-      fail("should throw InvalidSchemaException when GroupType contains no child");
-    } catch (InvalidSchemaException e) {
-      assertEquals("A group type can not be empty. Parquet does not support empty group without
leaves. Empty group: g1", e.getMessage());
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
b/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
index 4230561..5fac7bc 100644
--- a/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
+++ b/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
 
@@ -155,39 +154,20 @@ public class TestTypeBuilders {
 
   @Test
   public void testEmptyGroup() {
-    assertThrows("Should complain that required group is empty",
-        IllegalStateException.class, new Callable<Type>() {
-          @Override
-          public Type call() throws Exception {
-            return Types.requiredGroup().named("g");
-          }
-        });
-    assertThrows("Should complain that optional group is empty",
-        IllegalStateException.class, new Callable<Type>() {
-          @Override
-          public Type call() throws Exception {
-            return Types.optionalGroup().named("g");
-          }
-        });
-    assertThrows("Should complain that repeated group is empty",
-        IllegalStateException.class, new Callable<Type>() {
-          @Override
-          public Type call() throws Exception {
-            return Types.repeatedGroup().named("g");
-          }
-        });
+    // empty groups are allowed to support selecting 0 columns (counting rows)
+    Assert.assertEquals("Should not complain about an empty required group",
+        Types.requiredGroup().named("g"), new GroupType(REQUIRED, "g"));
+    Assert.assertEquals("Should not complain about an empty required group",
+        Types.optionalGroup().named("g"), new GroupType(OPTIONAL, "g"));
+    Assert.assertEquals("Should not complain about an empty required group",
+        Types.repeatedGroup().named("g"), new GroupType(REPEATED, "g"));
   }
 
   @Test
-  @Ignore(value="Enforcing this breaks tests in parquet-thrift")
   public void testEmptyMessage() {
-    assertThrows("Should complain that message is empty",
-        IllegalStateException.class, new Callable<Type>() {
-          @Override
-          public Type call() throws Exception {
-            return Types.buildMessage().named("m");
-          }
-        });
+    // empty groups are allowed to support selecting 0 columns (counting rows)
+    Assert.assertEquals("Should not complain about an empty required group",
+        Types.buildMessage().named("m"), new MessageType("m"));
   }
 
   @Test(expected=IllegalArgumentException.class)

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeUtil.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeUtil.java b/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeUtil.java
new file mode 100644
index 0000000..12ee3e1
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeUtil.java
@@ -0,0 +1,89 @@
+/**
+ * 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.schema;
+
+import org.junit.Test;
+
+import java.util.concurrent.Callable;
+
+import static org.apache.parquet.schema.OriginalType.UTF8;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.Type.Repetition.REPEATED;
+
+public class TestTypeUtil {
+  @Test
+  public void testWriteCheckMessageType() {
+    TypeUtil.checkValidWriteSchema(Types.buildMessage()
+        .required(INT32).named("a")
+        .optional(BINARY).as(UTF8).named("b")
+        .named("valid_schema"));
+
+    TestTypeBuilders.assertThrows("Should complain about empty MessageType",
+        InvalidSchemaException.class,
+        new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            TypeUtil.checkValidWriteSchema(new MessageType("invalid_schema"));
+            return null;
+          }
+        });
+  }
+
+  @Test
+  public void testWriteCheckGroupType() {
+    TypeUtil.checkValidWriteSchema(Types.repeatedGroup()
+        .required(INT32).named("a")
+        .optional(BINARY).as(UTF8).named("b")
+        .named("valid_group"));
+
+    TestTypeBuilders.assertThrows("Should complain about empty GroupType",
+        InvalidSchemaException.class,
+        new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            TypeUtil.checkValidWriteSchema(
+                new GroupType(REPEATED, "invalid_group"));
+            return null;
+          }
+        });
+  }
+
+  @Test
+  public void testWriteCheckNestedGroupType() {
+    TypeUtil.checkValidWriteSchema(Types.buildMessage()
+        .repeatedGroup()
+            .required(INT32).named("a")
+            .optional(BINARY).as(UTF8).named("b")
+            .named("valid_group")
+        .named("valid_message"));
+
+    TestTypeBuilders.assertThrows("Should complain about empty GroupType",
+        InvalidSchemaException.class,
+        new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            TypeUtil.checkValidWriteSchema(Types.buildMessage()
+                .addField(new GroupType(REPEATED, "invalid_group"))
+                .named("invalid_message"));
+            return null;
+          }
+        });
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
index 5f265e0..d6d8369 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
@@ -57,6 +57,7 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.io.ParquetEncodingException;
 import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+import org.apache.parquet.schema.TypeUtil;
 
 /**
  * Internal implementation of the Parquet file writer as a block container
@@ -205,6 +206,7 @@ public class ParquetFileWriter {
                            Path file, Mode mode, long rowGroupSize,
                            int maxPaddingSize)
       throws IOException {
+    TypeUtil.checkValidWriteSchema(schema);
     this.schema = schema;
     FileSystem fs = file.getFileSystem(configuration);
     boolean overwriteFlag = (mode == Mode.OVERWRITE);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
new file mode 100644
index 0000000..c63be91
--- /dev/null
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
@@ -0,0 +1,93 @@
+/**
+ * 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.hadoop.example;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.schema.MessageType;
+import java.io.IOException;
+
+/**
+ * An example file writer class.
+ * THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE.
+ */
+public class ExampleParquetWriter extends ParquetWriter<Group> {
+
+  /**
+   * Creates a Builder for configuring ParquetWriter with the example object
+   * model. THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE.
+   *
+   * @param file the output file to create
+   * @return a {@link Builder} to create a {@link ParquetWriter}
+   */
+  public static Builder builder(Path file) {
+    return new Builder(file);
+  }
+
+  /**
+   * Create a new {@link ExampleParquetWriter}.
+   *
+   * @param file The file name to write to.
+   * @param writeSupport The schema to write with.
+   * @param compressionCodecName Compression code to use, or CompressionCodecName.UNCOMPRESSED
+   * @param blockSize the block size threshold.
+   * @param pageSize See parquet write up. Blocks are subdivided into pages for alignment
and other purposes.
+   * @param enableDictionary Whether to use a dictionary to compress columns.
+   * @param conf The Configuration to use.
+   * @throws IOException
+   */
+  ExampleParquetWriter(Path file, WriteSupport<Group> writeSupport,
+                       CompressionCodecName compressionCodecName,
+                       int blockSize, int pageSize, boolean enableDictionary,
+                       boolean enableValidation,
+                       ParquetProperties.WriterVersion writerVersion,
+                       Configuration conf)
+      throws IOException {
+    super(file, writeSupport, compressionCodecName, blockSize, pageSize,
+        pageSize, enableDictionary, enableValidation, writerVersion, conf);
+  }
+
+  public static class Builder extends ParquetWriter.Builder<Group, Builder> {
+    private MessageType type = null;
+
+    private Builder(Path file) {
+      super(file);
+    }
+
+    public Builder withType(MessageType type) {
+      this.type = type;
+      return this;
+    }
+
+    @Override
+    protected Builder self() {
+      return this;
+    }
+
+    @Override
+    protected WriteSupport<Group> getWriteSupport(Configuration conf) {
+      return new GroupWriteSupport(type);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/GroupWriteSupport.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/GroupWriteSupport.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/GroupWriteSupport.java
index 0eb0cb8..25f8fe5 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/GroupWriteSupport.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/GroupWriteSupport.java
@@ -45,12 +45,22 @@ public class GroupWriteSupport extends WriteSupport<Group> {
     return parseMessageType(checkNotNull(configuration.get(PARQUET_EXAMPLE_SCHEMA), PARQUET_EXAMPLE_SCHEMA));
   }
 
-  private MessageType schema;
+  private MessageType schema = null;
   private GroupWriter groupWriter;
 
+  public GroupWriteSupport() {
+  }
+
+  GroupWriteSupport(MessageType schema) {
+    this.schema = schema;
+  }
+
   @Override
   public org.apache.parquet.hadoop.api.WriteSupport.WriteContext init(Configuration configuration)
{
-    schema = getSchema(configuration);
+    // if present, prefer the schema passed to the constructor
+    if (schema == null) {
+      schema = getSchema(configuration);
+    }
     return new WriteContext(schema, new HashMap<String, String>());
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
index 3dcec30..e327643 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.hadoop;
 
 import static java.util.Arrays.asList;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.apache.parquet.column.Encoding.DELTA_BYTE_ARRAY;
@@ -33,11 +34,21 @@ import static org.apache.parquet.hadoop.TestUtils.enforceEmptyDir;
 import static org.apache.parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
 import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
 
+import java.io.File;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.Callable;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.InvalidSchemaException;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 
 import org.apache.parquet.column.Encoding;
@@ -51,6 +62,7 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.MessageType;
+import org.junit.rules.TemporaryFolder;
 
 public class TestParquetWriter {
 
@@ -126,4 +138,29 @@ public class TestParquetWriter {
       }
     }
   }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Test
+  public void testBadWriteSchema() throws IOException {
+    final File file = temp.newFile("test.parquet");
+    file.delete();
+
+    TestUtils.assertThrows("Should reject a schema with an empty group",
+        InvalidSchemaException.class, new Callable<Void>() {
+          @Override
+          public Void call() throws IOException {
+            ExampleParquetWriter.builder(new Path(file.toString()))
+                .withType(Types.buildMessage()
+                    .addField(new GroupType(REQUIRED, "invalid_group"))
+                    .named("invalid_message"))
+                .build();
+            return null;
+          }
+        });
+
+    Assert.assertFalse("Should not create a file when schema is rejected",
+        file.exists());
+  }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/f203d809/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestUtils.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestUtils.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestUtils.java
index fcfd517..7c5a186 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestUtils.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestUtils.java
@@ -19,10 +19,12 @@
 package org.apache.parquet.hadoop;
 
 import java.io.IOException;
+import java.util.concurrent.Callable;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
 
 public class TestUtils {
 
@@ -37,4 +39,21 @@ public class TestUtils {
       throw new IOException("can not create path " + path);
     }
   }
+
+  /**
+   * A convenience method to avoid a large number of @Test(expected=...) tests
+   * @param message A String message to describe this assertion
+   * @param expected An Exception class that the Runnable should throw
+   * @param callable A Callable that is expected to throw the exception
+   */
+  public static void assertThrows(
+      String message, Class<? extends Exception> expected, Callable callable) {
+    try {
+      callable.call();
+      Assert.fail("No exception was thrown (" + message + "), expected: " +
+          expected.getName());
+    } catch (Exception actual) {
+      Assert.assertEquals(message, expected, actual.getClass());
+    }
+  }
 }


Mime
View raw message