You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by dw...@apache.org on 2015/03/31 20:22:54 UTC

incubator-parquet-mr git commit: PARQUET-165: Add a new parquet-benchmark module

Repository: incubator-parquet-mr
Updated Branches:
  refs/heads/master b8f5d89e0 -> 4fea3ea69


PARQUET-165: Add a new parquet-benchmark module

PARQUET-165

This PR is an initial version of a new ``parquet-benchmark`` module that we can build upon. The module already contains some simple benchmarks for read/writes, we can discuss how we can make those more representative.

When run, various statistics will be printed for all the benchmarks in this module. For example, for the read benchmarks the output will look like:

```
# Run complete. Total time: 00:03:16

Benchmark                                                             Mode  Samples  Score   Error  Units
p.b.ReadBenchmarks.read1MRowsBS256MPS4MUncompressed                  thrpt        1  0.248 ±   NaN  ops/s
p.b.ReadBenchmarks.read1MRowsBS256MPS8MUncompressed                  thrpt        1  0.331 ±   NaN  ops/s
p.b.ReadBenchmarks.read1MRowsBS512MPS4MUncompressed                  thrpt        1  0.309 ±   NaN  ops/s
p.b.ReadBenchmarks.read1MRowsBS512MPS8MUncompressed                  thrpt        1  0.303 ±   NaN  ops/s
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeGZIP             thrpt        1  0.264 ±   NaN  ops/s
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeSNAPPY           thrpt        1  0.499 ±   NaN  ops/s
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeUncompressed     thrpt        1  0.360 ±   NaN  ops/s
p.b.ReadBenchmarks.read1MRowsBS256MPS4MUncompressed                   avgt        1  3.623 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS256MPS8MUncompressed                   avgt        1  3.162 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS512MPS4MUncompressed                   avgt        1  3.231 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS512MPS8MUncompressed                   avgt        1  2.583 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeGZIP              avgt        1  3.713 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeSNAPPY            avgt        1  2.055 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeUncompressed      avgt        1  2.904 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS256MPS4MUncompressed                 sample        1  2.772 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS256MPS8MUncompressed                 sample        1  2.538 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS512MPS4MUncompressed                 sample        1  2.496 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS512MPS8MUncompressed                 sample        1  2.416 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeGZIP            sample        1  3.712 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeSNAPPY          sample        1  1.772 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeUncompressed    sample        1  2.819 ±   NaN   s/op
p.b.ReadBenchmarks.read1MRowsBS256MPS4MUncompressed                     ss        1  2.416 ±   NaN      s
p.b.ReadBenchmarks.read1MRowsBS256MPS8MUncompressed                     ss        1  2.564 ±   NaN      s
p.b.ReadBenchmarks.read1MRowsBS512MPS4MUncompressed                     ss        1  2.547 ±   NaN      s
p.b.ReadBenchmarks.read1MRowsBS512MPS8MUncompressed                     ss        1  3.094 ±   NaN      s
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeGZIP                ss        1  3.689 ±   NaN      s
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeSNAPPY              ss        1  1.983 ±   NaN      s
p.b.ReadBenchmarks.read1MRowsDefaultBlockAndPageSizeUncompressed        ss        1  2.928 ±   NaN      s

```

Author: Nezih Yigitbasi <ny...@netflix.com>

Closes #104 from nezihyigitbasi/benchmark-module and squashes the following commits:

90c72f5 [Nezih Yigitbasi] Add a new parquet-benchmark module


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

Branch: refs/heads/master
Commit: 4fea3ea6997c8135bc18a2bff31dc0a54e7bd82d
Parents: b8f5d89
Author: Nezih Yigitbasi <ny...@netflix.com>
Authored: Tue Mar 31 11:23:42 2015 -0700
Committer: Daniel Weeks <dw...@netflix.com>
Committed: Tue Mar 31 11:23:42 2015 -0700

----------------------------------------------------------------------
 parquet-benchmarks/README.md                    |  38 +++++
 parquet-benchmarks/pom.xml                      | 113 +++++++++++++
 parquet-benchmarks/run.sh                       |  31 ++++
 .../parquet/benchmarks/BenchmarkConstants.java  |  42 +++++
 .../java/parquet/benchmarks/BenchmarkFiles.java |  40 +++++
 .../java/parquet/benchmarks/BenchmarkUtils.java |  46 ++++++
 .../java/parquet/benchmarks/DataGenerator.java  | 144 +++++++++++++++++
 .../java/parquet/benchmarks/ReadBenchmarks.java | 106 +++++++++++++
 .../parquet/benchmarks/WriteBenchmarks.java     | 159 +++++++++++++++++++
 pom.xml                                         |   1 +
 10 files changed, 720 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/README.md
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/README.md b/parquet-benchmarks/README.md
new file mode 100644
index 0000000..646e6ba
--- /dev/null
+++ b/parquet-benchmarks/README.md
@@ -0,0 +1,38 @@
+<!--
+  ~ 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.
+  -->
+  
+##Running Parquet Benchmarks
+
+First, build the ``parquet-benchmarks`` module
+
+```
+mvn --projects parquet-benchmarks -amd -DskipTests -Denforcer.skip=true -P hadoop-2 clean package
+```
+
+Then, you can run all the benchmarks with the following command
+
+```
+ ./parquet-benchmarks/run.sh -wi 5 -i 5 -f 3 -bm all
+```
+
+To understand what each command line argument means and for more arguments please see
+
+```
+java -jar parquet-benchmarks/target/parquet-benchmarks.jar -help
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml
new file mode 100644
index 0000000..c4ed87c
--- /dev/null
+++ b/parquet-benchmarks/pom.xml
@@ -0,0 +1,113 @@
+<!--
+  - 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <groupId>com.twitter</groupId>
+    <artifactId>parquet</artifactId>
+    <relativePath>../pom.xml</relativePath>
+    <version>1.6.0rc3-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-benchmarks</artifactId>
+  <packaging>jar</packaging>
+  <name>Parquet Benchmarks</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <properties>
+    <jmh.version>1.3.4</jmh.version>
+    <uberjar.name>parquet-benchmarks</uberjar.name>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-encoding</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+       <groupId>com.twitter</groupId>
+       <artifactId>parquet-hadoop</artifactId>
+       <version>${project.version}</version>
+    </dependency>
+    <dependency>
+       <groupId>org.apache.hadoop</groupId>
+       <artifactId>hadoop-client</artifactId>
+       <version>${hadoop.version}</version>
+    </dependency>
+    <dependency>
+       <groupId>org.openjdk.jmh</groupId>
+       <artifactId>jmh-core</artifactId>
+       <version>${jmh.version}</version>
+    </dependency>
+    <dependency>
+       <groupId>org.openjdk.jmh</groupId>
+        <artifactId>jmh-generator-annprocess</artifactId>
+        <version>${jmh.version}</version>
+        <scope>provided</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <compilerVersion>${targetJavaVersion}</compilerVersion>
+          <source>${targetJavaVersion}</source>
+          <target>${targetJavaVersion}</target>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <finalName>${uberjar.name}</finalName>
+              <!-- when minimized some runtime dependencies are also removed, so set minimizeJar = false -->
+              <minimizeJar>false</minimizeJar>
+              <transformers>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                  <mainClass>org.openjdk.jmh.Main</mainClass>
+                </transformer>
+              </transformers>
+              <artifactSet>
+                <includes>
+                  <include>*:*</include>
+                </includes>
+                <excludes>
+                  <exclude>META-INF/*.SF</exclude>
+                  <exclude>META-INF/*.DSA</exclude>
+                  <exclude>META-INF/*.RSA</exclude>
+                </excludes>
+              </artifactSet>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/run.sh
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/run.sh b/parquet-benchmarks/run.sh
new file mode 100755
index 0000000..e92b57d
--- /dev/null
+++ b/parquet-benchmarks/run.sh
@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+# !/usr/bin/env bash
+
+SCRIPT_PATH=$( cd "$(dirname "$0")" ; pwd -P )
+
+echo "Starting WRITE benchmarks"
+java -jar ${SCRIPT_PATH}/target/parquet-benchmarks.jar p*Write* "$@"
+echo "Generating test data"
+java -cp ${SCRIPT_PATH}/target/parquet-benchmarks.jar parquet.benchmarks.DataGenerator generate
+echo "Data generated, starting READ benchmarks"
+java -jar ${SCRIPT_PATH}/target/parquet-benchmarks.jar p*Read* "$@"
+echo "Cleaning up generated data"
+java -cp ${SCRIPT_PATH}/target/parquet-benchmarks.jar parquet.benchmarks.DataGenerator cleanup
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkConstants.java
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkConstants.java b/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkConstants.java
new file mode 100644
index 0000000..4f66ccb
--- /dev/null
+++ b/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkConstants.java
@@ -0,0 +1,42 @@
+/*
+ * 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.benchmarks;
+
+import static parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static parquet.hadoop.ParquetWriter.DEFAULT_PAGE_SIZE;
+
+public class BenchmarkConstants {
+  public static final int ONE_K = 1000;
+  public static final int FIVE_K = 5 * ONE_K;
+  public static final int TEN_K = 2 * FIVE_K;
+  public static final int HUNDRED_K = 10 * TEN_K;
+  public static final int ONE_MILLION = 10 * HUNDRED_K;
+
+  public static final int FIXED_LEN_BYTEARRAY_SIZE = 1024;
+
+  public static final int BLOCK_SIZE_DEFAULT = DEFAULT_BLOCK_SIZE;
+  public static final int BLOCK_SIZE_256M = 256 * 1024 * 1024;
+  public static final int BLOCK_SIZE_512M = 512 * 1024 * 1024;
+
+  public static final int PAGE_SIZE_DEFAULT = DEFAULT_PAGE_SIZE;
+  public static final int PAGE_SIZE_4M = 4 * 1024 * 1024;
+  public static final int PAGE_SIZE_8M = 8 * 1024 * 1024;
+
+  public static final int DICT_PAGE_SIZE = 512;
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkFiles.java
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkFiles.java b/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkFiles.java
new file mode 100644
index 0000000..1e57ca2
--- /dev/null
+++ b/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkFiles.java
@@ -0,0 +1,40 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+public class BenchmarkFiles {
+  public static final Configuration configuration = new Configuration();
+
+  public static final String TARGET_DIR = "target/tests/ParquetBenchmarks";
+  public static final Path file_1M = new Path(TARGET_DIR + "/PARQUET-1M");
+
+  //different block and page sizes
+  public static final Path file_1M_BS256M_PS4M = new Path(TARGET_DIR + "/PARQUET-1M-BS256M_PS4M");
+  public static final Path file_1M_BS256M_PS8M = new Path(TARGET_DIR + "/PARQUET-1M-BS256M_PS8M");
+  public static final Path file_1M_BS512M_PS4M = new Path(TARGET_DIR + "/PARQUET-1M-BS512M_PS4M");
+  public static final Path file_1M_BS512M_PS8M = new Path(TARGET_DIR + "/PARQUET-1M-BS512M_PS8M");
+
+  //different compression codecs
+//  public final Path parquetFile_1M_LZO = new Path("target/tests/ParquetBenchmarks/PARQUET-1M-LZO");
+  public static final Path file_1M_SNAPPY = new Path(TARGET_DIR + "/PARQUET-1M-SNAPPY");
+  public static final Path file_1M_GZIP = new Path(TARGET_DIR + "/PARQUET-1M-GZIP");
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkUtils.java
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkUtils.java b/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkUtils.java
new file mode 100644
index 0000000..9400bc1
--- /dev/null
+++ b/parquet-benchmarks/src/main/java/parquet/benchmarks/BenchmarkUtils.java
@@ -0,0 +1,46 @@
+/*
+ * 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.benchmarks;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class BenchmarkUtils {
+  public static void deleteIfExists(Configuration conf, Path path) {
+    try {
+      FileSystem fs = path.getFileSystem(conf);
+      if (fs.exists(path)) {
+        if (!fs.delete(path, true)) {
+          System.err.println("Couldn't delete " + path);
+        }
+      }
+    } catch (IOException e) {
+      System.err.println("Couldn't delete " + path);
+      e.printStackTrace();
+    }
+  }
+
+  public static boolean exists(Configuration conf, Path path) throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+    return fs.exists(path);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/src/main/java/parquet/benchmarks/DataGenerator.java
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/src/main/java/parquet/benchmarks/DataGenerator.java b/parquet-benchmarks/src/main/java/parquet/benchmarks/DataGenerator.java
new file mode 100644
index 0000000..f1af4f9
--- /dev/null
+++ b/parquet-benchmarks/src/main/java/parquet/benchmarks/DataGenerator.java
@@ -0,0 +1,144 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import parquet.column.ParquetProperties;
+import parquet.example.data.Group;
+import parquet.example.data.simple.SimpleGroupFactory;
+import parquet.hadoop.ParquetWriter;
+import parquet.hadoop.example.GroupWriteSupport;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.io.api.Binary;
+import parquet.schema.MessageType;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import static java.util.UUID.randomUUID;
+import static parquet.benchmarks.BenchmarkUtils.deleteIfExists;
+import static parquet.benchmarks.BenchmarkUtils.exists;
+import static parquet.column.ParquetProperties.WriterVersion.PARQUET_2_0;
+import static parquet.hadoop.metadata.CompressionCodecName.GZIP;
+import static parquet.hadoop.metadata.CompressionCodecName.SNAPPY;
+import static parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
+import static parquet.schema.MessageTypeParser.parseMessageType;
+import static parquet.benchmarks.BenchmarkConstants.*;
+import static parquet.benchmarks.BenchmarkFiles.*;
+
+public class DataGenerator {
+
+  public void generateAll() {
+    try {
+      generateData(file_1M, configuration, PARQUET_2_0, BLOCK_SIZE_DEFAULT, PAGE_SIZE_DEFAULT, FIXED_LEN_BYTEARRAY_SIZE, UNCOMPRESSED, ONE_MILLION);
+
+      //generate data for different block and page sizes
+      generateData(file_1M_BS256M_PS4M, configuration, PARQUET_2_0, BLOCK_SIZE_256M, PAGE_SIZE_4M, FIXED_LEN_BYTEARRAY_SIZE, UNCOMPRESSED, ONE_MILLION);
+      generateData(file_1M_BS256M_PS8M, configuration, PARQUET_2_0, BLOCK_SIZE_256M, PAGE_SIZE_8M, FIXED_LEN_BYTEARRAY_SIZE, UNCOMPRESSED, ONE_MILLION);
+      generateData(file_1M_BS512M_PS4M, configuration, PARQUET_2_0, BLOCK_SIZE_512M, PAGE_SIZE_4M, FIXED_LEN_BYTEARRAY_SIZE, UNCOMPRESSED, ONE_MILLION);
+      generateData(file_1M_BS512M_PS8M, configuration, PARQUET_2_0, BLOCK_SIZE_512M, PAGE_SIZE_8M, FIXED_LEN_BYTEARRAY_SIZE, UNCOMPRESSED, ONE_MILLION);
+
+      //generate data for different codecs
+//      generateData(parquetFile_1M_LZO, configuration, PARQUET_2_0, BLOCK_SIZE_DEFAULT, PAGE_SIZE_DEFAULT, FIXED_LEN_BYTEARRAY_SIZE, LZO, ONE_MILLION);
+      generateData(file_1M_SNAPPY, configuration, PARQUET_2_0, BLOCK_SIZE_DEFAULT, PAGE_SIZE_DEFAULT, FIXED_LEN_BYTEARRAY_SIZE, SNAPPY, ONE_MILLION);
+      generateData(file_1M_GZIP, configuration, PARQUET_2_0, BLOCK_SIZE_DEFAULT, PAGE_SIZE_DEFAULT, FIXED_LEN_BYTEARRAY_SIZE, GZIP, ONE_MILLION);
+    }
+    catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void generateData(Path outFile, Configuration configuration, ParquetProperties.WriterVersion version,
+                           int blockSize, int pageSize, int fixedLenByteArraySize, CompressionCodecName codec, int nRows)
+          throws IOException
+  {
+    if (exists(configuration, outFile)) {
+      System.out.println("File already exists " + outFile);
+      return;
+    }
+
+    System.out.println("Generating data @ " + outFile);
+
+    MessageType schema = parseMessageType(
+            "message test { "
+                    + "required binary binary_field; "
+                    + "required int32 int32_field; "
+                    + "required int64 int64_field; "
+                    + "required boolean boolean_field; "
+                    + "required float float_field; "
+                    + "required double double_field; "
+                    + "required fixed_len_byte_array(" + fixedLenByteArraySize +") flba_field; "
+                    + "required int96 int96_field; "
+                    + "} ");
+
+    GroupWriteSupport.setSchema(schema, configuration);
+    SimpleGroupFactory f = new SimpleGroupFactory(schema);
+    ParquetWriter<Group> writer = new ParquetWriter<Group>(outFile, new GroupWriteSupport(), codec, blockSize,
+                                                           pageSize, DICT_PAGE_SIZE, true, false, version, configuration);
+
+    //generate some data for the fixed len byte array field
+    char[] chars = new char[fixedLenByteArraySize];
+    Arrays.fill(chars, '*');
+
+    for (int i = 0; i < nRows; i++) {
+      writer.write(
+        f.newGroup()
+          .append("binary_field", randomUUID().toString())
+          .append("int32_field", i)
+          .append("int64_field", 64l)
+          .append("boolean_field", true)
+          .append("float_field", 1.0f)
+          .append("double_field", 2.0d)
+          .append("flba_field", new String(chars))
+          .append("int96_field", Binary.fromByteArray(new byte[12]))
+      );
+    }
+    writer.close();
+  }
+
+  public void cleanup()
+  {
+    deleteIfExists(configuration, file_1M);
+    deleteIfExists(configuration, file_1M_BS256M_PS4M);
+    deleteIfExists(configuration, file_1M_BS256M_PS8M);
+    deleteIfExists(configuration, file_1M_BS512M_PS4M);
+    deleteIfExists(configuration, file_1M_BS512M_PS8M);
+//    deleteIfExists(configuration, parquetFile_1M_LZO);
+    deleteIfExists(configuration, file_1M_SNAPPY);
+    deleteIfExists(configuration, file_1M_GZIP);
+  }
+
+  public static void main(String[] args) {
+    DataGenerator generator = new DataGenerator();
+    if (args.length < 1) {
+      System.err.println("Please specify a command (generate|cleanup).");
+      System.exit(1);
+    }
+
+    String command = args[0];
+    if (command.equalsIgnoreCase("generate")) {
+      generator.generateAll();
+    } else if (command.equalsIgnoreCase("cleanup")) {
+      generator.cleanup();
+    } else {
+      throw new IllegalArgumentException("invalid command " + command);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/src/main/java/parquet/benchmarks/ReadBenchmarks.java
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/src/main/java/parquet/benchmarks/ReadBenchmarks.java b/parquet-benchmarks/src/main/java/parquet/benchmarks/ReadBenchmarks.java
new file mode 100644
index 0000000..e308e88
--- /dev/null
+++ b/parquet-benchmarks/src/main/java/parquet/benchmarks/ReadBenchmarks.java
@@ -0,0 +1,106 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.hadoop.fs.Path;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.infra.Blackhole;
+import parquet.example.data.Group;
+import parquet.hadoop.ParquetReader;
+import parquet.hadoop.example.GroupReadSupport;
+import static parquet.benchmarks.BenchmarkConstants.*;
+import static parquet.benchmarks.BenchmarkFiles.*;
+
+import java.io.IOException;
+
+public class ReadBenchmarks {
+  private void read(Path parquetFile, int nRows, Blackhole blackhole) throws IOException
+  {
+    ParquetReader<Group> reader = ParquetReader.builder(new GroupReadSupport(), parquetFile).withConf(configuration).build();
+    for (int i = 0; i < nRows; i++) {
+      Group group = reader.read();
+      blackhole.consume(group.getBinary("binary_field", 0));
+      blackhole.consume(group.getInteger("int32_field", 0));
+      blackhole.consume(group.getLong("int64_field", 0));
+      blackhole.consume(group.getBoolean("boolean_field", 0));
+      blackhole.consume(group.getFloat("float_field", 0));
+      blackhole.consume(group.getDouble("double_field", 0));
+      blackhole.consume(group.getBinary("flba_field", 0));
+      blackhole.consume(group.getInt96("int96_field", 0));
+    }
+    reader.close();
+  }
+
+  @Benchmark
+  public void read1MRowsDefaultBlockAndPageSizeUncompressed(Blackhole blackhole)
+          throws IOException
+  {
+    read(file_1M, ONE_MILLION, blackhole);
+  }
+
+  @Benchmark
+  public void read1MRowsBS256MPS4MUncompressed(Blackhole blackhole)
+          throws IOException
+  {
+    read(file_1M_BS256M_PS4M, ONE_MILLION, blackhole);
+  }
+
+  @Benchmark
+  public void read1MRowsBS256MPS8MUncompressed(Blackhole blackhole)
+          throws IOException
+  {
+    read(file_1M_BS256M_PS8M, ONE_MILLION, blackhole);
+  }
+
+  @Benchmark
+  public void read1MRowsBS512MPS4MUncompressed(Blackhole blackhole)
+          throws IOException
+  {
+    read(file_1M_BS512M_PS4M, ONE_MILLION, blackhole);
+  }
+
+  @Benchmark
+  public void read1MRowsBS512MPS8MUncompressed(Blackhole blackhole)
+          throws IOException
+  {
+    read(file_1M_BS512M_PS8M, ONE_MILLION, blackhole);
+  }
+
+  //TODO how to handle lzo jar?
+//  @Benchmark
+//  public void read1MRowsDefaultBlockAndPageSizeLZO(Blackhole blackhole)
+//          throws IOException
+//  {
+//    read(parquetFile_1M_LZO, ONE_MILLION, blackhole);
+//  }
+
+  @Benchmark
+  public void read1MRowsDefaultBlockAndPageSizeSNAPPY(Blackhole blackhole)
+          throws IOException
+  {
+    read(file_1M_SNAPPY, ONE_MILLION, blackhole);
+  }
+
+  @Benchmark
+  public void read1MRowsDefaultBlockAndPageSizeGZIP(Blackhole blackhole)
+          throws IOException
+  {
+    read(file_1M_GZIP, ONE_MILLION, blackhole);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/parquet-benchmarks/src/main/java/parquet/benchmarks/WriteBenchmarks.java
----------------------------------------------------------------------
diff --git a/parquet-benchmarks/src/main/java/parquet/benchmarks/WriteBenchmarks.java b/parquet-benchmarks/src/main/java/parquet/benchmarks/WriteBenchmarks.java
new file mode 100644
index 0000000..ca6e43b
--- /dev/null
+++ b/parquet-benchmarks/src/main/java/parquet/benchmarks/WriteBenchmarks.java
@@ -0,0 +1,159 @@
+/*
+ * 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.benchmarks;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+
+import static org.openjdk.jmh.annotations.Scope.Thread;
+import static parquet.benchmarks.BenchmarkConstants.*;
+import static parquet.benchmarks.BenchmarkFiles.*;
+
+import java.io.IOException;
+
+import static parquet.column.ParquetProperties.WriterVersion.PARQUET_2_0;
+import static parquet.hadoop.metadata.CompressionCodecName.GZIP;
+import static parquet.hadoop.metadata.CompressionCodecName.SNAPPY;
+import static parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
+
+@State(Thread)
+public class WriteBenchmarks {
+  private DataGenerator dataGenerator = new DataGenerator();
+
+  @Setup(Level.Iteration)
+  public void cleanup() {
+    //clean existing test data at the beginning of each iteration
+    dataGenerator.cleanup();
+  }
+
+  @Benchmark
+  public void write1MRowsDefaultBlockAndPageSizeUncompressed()
+          throws IOException
+  {
+    dataGenerator.generateData(file_1M,
+                               configuration,
+                               PARQUET_2_0,
+                               BLOCK_SIZE_DEFAULT,
+                               PAGE_SIZE_DEFAULT,
+                               FIXED_LEN_BYTEARRAY_SIZE,
+                               UNCOMPRESSED,
+                               ONE_MILLION);
+  }
+
+  @Benchmark
+  public void write1MRowsBS256MPS4MUncompressed()
+          throws IOException
+  {
+    dataGenerator.generateData(file_1M_BS256M_PS4M,
+                               configuration,
+                               PARQUET_2_0,
+                               BLOCK_SIZE_256M,
+                               PAGE_SIZE_4M,
+                               FIXED_LEN_BYTEARRAY_SIZE,
+                               UNCOMPRESSED,
+                               ONE_MILLION);
+  }
+
+  @Benchmark
+  public void write1MRowsBS256MPS8MUncompressed()
+          throws IOException
+  {
+    dataGenerator.generateData(file_1M_BS256M_PS8M,
+                               configuration,
+                               PARQUET_2_0,
+                               BLOCK_SIZE_256M,
+                               PAGE_SIZE_8M,
+                               FIXED_LEN_BYTEARRAY_SIZE,
+                               UNCOMPRESSED,
+                               ONE_MILLION);
+  }
+
+  @Benchmark
+  public void write1MRowsBS512MPS4MUncompressed()
+          throws IOException
+  {
+    dataGenerator.generateData(file_1M_BS512M_PS4M,
+                               configuration,
+                               PARQUET_2_0,
+                               BLOCK_SIZE_512M,
+                               PAGE_SIZE_4M,
+                               FIXED_LEN_BYTEARRAY_SIZE,
+                               UNCOMPRESSED,
+                               ONE_MILLION);
+  }
+
+  @Benchmark
+  public void write1MRowsBS512MPS8MUncompressed()
+          throws IOException
+  {
+    dataGenerator.generateData(file_1M_BS512M_PS8M,
+                               configuration,
+                               PARQUET_2_0,
+                               BLOCK_SIZE_512M,
+                               PAGE_SIZE_8M,
+                               FIXED_LEN_BYTEARRAY_SIZE,
+                               UNCOMPRESSED,
+                               ONE_MILLION);
+  }
+
+  //TODO how to handle lzo jar?
+//  @Benchmark
+//  public void write1MRowsDefaultBlockAndPageSizeLZO()
+//          throws IOException
+//  {
+//    dataGenerator.generateData(parquetFile_1M_LZO,
+//            configuration,
+//            WriterVersion.PARQUET_2_0,
+//            BLOCK_SIZE_DEFAULT,
+//            PAGE_SIZE_DEFAULT,
+//            FIXED_LEN_BYTEARRAY_SIZE,
+//            LZO,
+//            ONE_MILLION);
+//  }
+
+  @Benchmark
+  public void write1MRowsDefaultBlockAndPageSizeSNAPPY()
+          throws IOException
+  {
+    dataGenerator.generateData(file_1M_SNAPPY,
+                               configuration,
+                               PARQUET_2_0,
+                               BLOCK_SIZE_DEFAULT,
+                               PAGE_SIZE_DEFAULT,
+                               FIXED_LEN_BYTEARRAY_SIZE,
+                               SNAPPY,
+                               ONE_MILLION);
+  }
+
+  @Benchmark
+  public void write1MRowsDefaultBlockAndPageSizeGZIP()
+          throws IOException
+  {
+    dataGenerator.generateData(file_1M_GZIP,
+                               configuration,
+                               PARQUET_2_0,
+                               BLOCK_SIZE_DEFAULT,
+                               PAGE_SIZE_DEFAULT,
+                               FIXED_LEN_BYTEARRAY_SIZE,
+                               GZIP,
+                               ONE_MILLION);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/4fea3ea6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1687f51..1354983 100644
--- a/pom.xml
+++ b/pom.xml
@@ -119,6 +119,7 @@
 
   <modules>
     <module>parquet-avro</module>
+    <module>parquet-benchmarks</module>
     <module>parquet-cascading</module>
     <module>parquet-column</module>
     <module>parquet-common</module>