Merge branch 'master' into fsa_codegen2

Conflicts:
	parquet-column/pom.xml
	parquet-column/src/main/java/parquet/example/DummyRecordConverter.java
	parquet-column/src/main/java/parquet/schema/PrimitiveType.java
	parquet-column/src/test/java/parquet/io/PerfTest.java
diff --git a/.gitignore b/.gitignore
index 39db6ba..c80c514 100644
--- a/.gitignore
+++ b/.gitignore
@@ -12,3 +12,5 @@
 *.iws
 *.orig
 *.rej
+dependency-reduced-pom.xml
+.idea/*
\ No newline at end of file
diff --git a/NOTICE b/NOTICE
index f3a6800..dc96637 100644
--- a/NOTICE
+++ b/NOTICE
@@ -10,3 +10,8 @@
 Hadoop 0.20.2
 Apache Public License 2.0
 http://hadoop.apache.org
+
+This project includes code from https://github.com/lemire/JavaFastPFOR
+parquet-column/src/main/java/parquet/column/values/bitpacking/LemireBitPacking.java
+Apache License Version 2.0 http://www.apache.org/licenses/.
+(c) Daniel Lemire, http://lemire.me/en/
diff --git a/README.md b/README.md
index 40e1750..e8081f2 100644
--- a/README.md
+++ b/README.md
@@ -1,39 +1,134 @@
 Parquet MR [![Build Status](https://travis-ci.org/Parquet/parquet-mr.png?branch=master)](http://travis-ci.org/Parquet/parquet-mr)
 ======
 
-Parquet-mr is the java implementation of the [Parquet format](https://github.com/Parquet/parquet-format) to be used in Hadoop. 
-It uses the [record shredding and assembly algorithm](https://github.com/Parquet/parquet-mr/wiki/The-striping-and-assembly-algorithms-from-the-Dremel-paper) described in the Dremel paper.
-Integration with Pig and Map/Reduce are provided.
+Parquet-MR contains the java implementation of the [Parquet format](https://github.com/Parquet/parquet-format). 
+Parquet is a columnar storage format for Hadoop; it provides efficient storage and encoding of data.
+Parquet uses the [record shredding and assembly algorithm](https://github.com/Parquet/parquet-mr/wiki/The-striping-and-assembly-algorithms-from-the-Dremel-paper) described in the Dremel paper to represent nested structures.
+
+You can find some details about the format and intended use cases in our [Hadoop Summit 2013 presentation](http://www.slideshare.net/julienledem/parquet-hadoop-summit-2013)
+
+## Features
+
+Parquet is a very active project, and new features are being added quickly; below is the state as of June 2013.
+
+
+<table>
+  <tr><th>Feature</th><th>In trunk</th><th>In dev</th><th>Planned</th><th>Expected release</th></tr>
+  <tr><td>Type-specific encoding</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Hive integration</td><td></td><td>YES (<a href ="https://github.com/Parquet/parquet-mr/pull/28">28</a>)</td></td><td></td><td>1.0</td></tr>
+  <tr><td>Pig integration</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Cascading integration</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Impala integration</td><td>YES (non-nested)</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Java Map/Reduce API</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Native Avro support</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Native Thrift support</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Complex structure support</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Future-proofed versioning</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>RLE</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Bit Packing</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Adaptive dictionary encoding</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Complex structure support</td><td>YES</td><td></td></td><td></td><td>1.0</td></tr>
+  <tr><td>Predicate pushdown</td><td></td><td>YES (<a href ="https://github.com/Parquet/parquet-mr/pull/68">68</a>)</td></td><td></td><td>2.0</td></tr>
+  <tr><td>Column stats</td><td></td><td></td></td><td>YES</td><td>2.0</td></tr>  <tr><td>Delta encoding</td><td></td><td></td></td><td>YES</td><td>2.0</td></tr>
+  <tr><td>Native Protocol Buffers support</td><td></td><td></td></td><td>YES</td><td>2.0</td></tr>
+  <tr><td>Index pages</td><td></td><td></td></td><td>YES</td><td>2.0</td></tr>
+</table>
+
+## Map/Reduce integration
+
+[Input](https://github.com/Parquet/parquet-mr/blob/master/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java) and [Output](https://github.com/Parquet/parquet-mr/blob/master/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java) formats.
+Note that to use an Input or Output format, you need to implement a WriteSupport or ReadSupport class, which will implement the conversion of your object to and from a Parquet schema.
+
+We've implemented this for 2 popular data formats to provide a clean migration path as well:
+
+### Thrift
+Thrift integration is provided by the [parquet-thrift](https://github.com/Parquet/parquet-mr/tree/master/parquet-thrift) sub-project. If you are using Thrift through Scala, you may be using Twitter's [Scrooge](https://github.com/twitter/scrooge). If that's the case, not to worry -- we took care of the Scrooge/Apache Thrift glue for you in the [parquet-scrooge](https://github.com/Parquet/parquet-mr/tree/master/parquet-scrooge) sub-project.
+
+### Avro
+Avro conversion is implemented via the [parquet-avro](https://github.com/Parquet/parquet-mr/tree/master/parquet-avro) sub-project.
+
+### Create your own objects
+* The ParquetOutputFormat can be provided a WriteSupport to write your own objects to an event based RecordConsumer.
+* the ParquetInputFormat can be provided a ReadSupport to materialize your own objects by implementing a RecordMaterializer
+
+See the APIs:
+* [Record conversion API](https://github.com/Parquet/parquet-mr/tree/master/parquet-column/src/main/java/parquet/io/api)
+* [Hadoop API](https://github.com/Parquet/parquet-mr/tree/master/parquet-hadoop/src/main/java/parquet/hadoop/api)
 
 ## Apache Pig integration
 A [Loader](https://github.com/Parquet/parquet-mr/blob/master/parquet-pig/src/main/java/parquet/pig/ParquetLoader.java) and a [Storer](https://github.com/Parquet/parquet-mr/blob/master/parquet-pig/src/main/java/parquet/pig/ParquetStorer.java) are provided to read and write Parquet files with Apache Pig
 
-## Map/Reduce integration
+Storing data into Parquet in Pig is simple:
+```
+-- options you might want to fiddle with
+SET parquet.page.size 1048576 -- default. this is your min read/write unit.
+SET parquet.block.size 524288000 -- your memory budget for buffering data
+SET parquet.compression lzo -- or you can use none, gzip, snappy
+STORE mydata into '/some/path' USING parquet.pig.ParquetStorer;
+```
+Reading in Pig is also simple:
+```
+mydata = LOAD '/some/path' USING parquet.pig.ParquetLoader();
+```
 
-### Thrift
-Thrift mapping to the parquet schema is provided using a TBase extending class.
-You can read and write parquet files using Thrift generated classes.
+If the data was stored using Pig, things will "just work". If the data was stored using another method, you will need to provide the Pig schema equivalent to the data you stored (you can also write the schema to the file footer while writing it -- but that's pretty advanced). We will provide a basic automatic schema conversion soon.
 
-### Create your own objects
-* The ParquetOutputFormat can be provided a WriteSupport to write your own objects to an event based RecordConsumer.
-* the ParquetInputFormat can be provided a ReadSupport to materialize your own POJOs by implementing a RecordMaterializer
+## Hive integration
 
-See the APIs:
-* [Record convertion API](https://github.com/Parquet/parquet-mr/tree/master/parquet-column/src/main/java/parquet/io/api)
-* [Hadoop API](https://github.com/Parquet/parquet-mr/tree/master/parquet-hadoop/src/main/java/parquet/hadoop/api)
+Hive integration is under development in <a href ="https://github.com/Parquet/parquet-mr/pull/28">Issue 28</a> and very close to being merged! Feel free to try out that branch and tell us what you think.
 
 ## Build
 
 to run the unit tests:
 mvn test
 
+to build the jars:
+mvn package
+
 The build runs in [Travis CI](http://travis-ci.org/Parquet/parquet-mr):
-[![Build Status](https://secure.travis-ci.org/Parquet/parquet-mr.png)](http://travis-ci.org/Parquet/parquet-mr)
+[![Build Status](https://secure.travis-ci.org/Parquet/parquet-mr.png?branch=master)](http://travis-ci.org/Parquet/parquet-mr)
+
+## Add Parquet as a dependency in Maven
+
+### Snapshot releases
+```xml
+  <repositories>
+    <repository>
+      <id>sonatype-nexus-snapshots</id>
+      <url>https://oss.sonatype.org/content/repositories/snapshots</url>
+      <releases>
+        <enabled>false</enabled>
+      </releases>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+     </repository>
+  </repositories>
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>1.0.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>1.0.0-SNAPSHOT</version>
+    </dependency>
+  </dependencies>
+```
+
+### Official releases
+We haven't published a 1.0.0 yet
 
 ## Authors and contributors
 
-* Julien Le Dem <http://twitter.com/J_>
+* Julien Le Dem [@J_](http://twitter.com/J_) <https://github.com/julienledem>
+* Tom White <https://github.com/tomwhite>
+* Avi Bryant <https://github.com/avibryant>
+* Dmitriy Ryaboy [@squarecog](https://twitter.com/squarecog) <https://github.com/dvryaboy>
 * Jonathan Coveney <http://twitter.com/jco>
+* and many others -- see the [Contributor report]( https://github.com/Parquet/parquet-mr/contributors)
 
 ## Discussions
 * google group https://groups.google.com/d/forum/parquet-dev
@@ -41,7 +136,7 @@
 
 ## License
 
-Copyright 2012 Twitter, Inc.
+Copyright 2012-2013 Twitter, Inc.
 
 Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0
 
diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml
new file mode 100644
index 0000000..3a03213
--- /dev/null
+++ b/parquet-avro/pom.xml
@@ -0,0 +1,152 @@
+<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.0.0-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-avro</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Parquet Avro</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <properties>
+    <avro.version>1.7.4</avro.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-format</artifactId>
+      <version>1.0.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+      <version>${avro.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>11.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.0</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <artifactSet>
+                <includes>
+                  <include>org.codehaus.jackson:jackson-mapper-asl</include>
+                  <include>org.codehaus.jackson:jackson-core-asl</include>
+                </includes>
+              </artifactSet>
+              <relocations>
+                <relocation>
+                  <pattern>org.codehaus.jackson</pattern>
+                  <shadedPattern>parquet.org.codehaus.jackson</shadedPattern>
+                </relocation>
+              </relocations>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.avro</groupId>
+        <artifactId>avro-maven-plugin</artifactId>
+        <version>${avro.version}</version>
+        <executions>
+            <execution>
+                <phase>generate-test-sources</phase>
+                <goals>
+                    <goal>idl-protocol</goal>
+                </goals>
+                <configuration>
+                    <sourceDirectory>${project.basedir}/src/test/resources</sourceDirectory>
+                    <outputDirectory>${project.build.directory}/generated-test-sources</outputDirectory>
+                    <stringType>String</stringType>
+                </configuration>
+            </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <!-- Ensure that the specific classes are available during test compile but not included in jar -->
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.8</version>
+        <executions>
+          <execution>
+            <id>add-test-sources</id>
+            <phase>generate-test-sources</phase>
+            <goals>
+              <goal>add-test-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>${project.build.directory}/generated-test-sources</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java b/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java
new file mode 100644
index 0000000..b2a2805
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java
@@ -0,0 +1,448 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericArray;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.specific.SpecificData;
+import parquet.Preconditions;
+import parquet.io.api.Binary;
+import parquet.io.api.Converter;
+import parquet.io.api.GroupConverter;
+import parquet.io.api.PrimitiveConverter;
+import parquet.schema.GroupType;
+import parquet.schema.MessageType;
+import parquet.schema.Type;
+
+class AvroIndexedRecordConverter extends GroupConverter {
+
+  private final ParentValueContainer parent;
+  protected IndexedRecord currentRecord;
+  private final Converter[] converters;
+
+  private final GroupType parquetSchema;
+  private final Schema avroSchema;
+  private final Class<? extends IndexedRecord> specificClass;
+
+  public AvroIndexedRecordConverter(MessageType parquetSchema, Schema avroSchema) {
+    this(null, parquetSchema, avroSchema);
+  }
+
+  public AvroIndexedRecordConverter(ParentValueContainer parent, GroupType
+      parquetSchema, Schema avroSchema) {
+    this.parent = parent;
+    this.parquetSchema = parquetSchema;
+    this.avroSchema = avroSchema;
+    int schemaSize = parquetSchema.getFieldCount();
+    this.converters = new Converter[schemaSize];
+    this.specificClass = SpecificData.get().getClass(avroSchema);
+    int index = 0; // parquet ignores Avro nulls, so index may differ
+    for (int avroIndex = 0; avroIndex < avroSchema.getFields().size(); avroIndex++) {
+      Schema.Field field = avroSchema.getFields().get(avroIndex);
+      if (field.schema().getType().equals(Schema.Type.NULL)) {
+        continue;
+      }
+      Type type = parquetSchema.getType(index);
+      final int finalAvroIndex = avroIndex;
+      final Schema fieldSchema = AvroSchemaConverter.getNonNull(field.schema());
+      converters[index] = newConverter(fieldSchema, type, new ParentValueContainer() {
+        @Override
+        void add(Object value) {
+          AvroIndexedRecordConverter.this.set(finalAvroIndex, value);
+        }
+      });
+      index++;
+    }
+  }
+
+  private static Converter newConverter(Schema schema, Type type,
+      ParentValueContainer parent) {
+    if (schema.getType().equals(Schema.Type.BOOLEAN)) {
+      return new FieldBooleanConverter(parent);
+    } else if (schema.getType().equals(Schema.Type.INT)) {
+      return new FieldIntegerConverter(parent);
+    } else if (schema.getType().equals(Schema.Type.LONG)) {
+      return new FieldLongConverter(parent);
+    } else if (schema.getType().equals(Schema.Type.FLOAT)) {
+      return new FieldFloatConverter(parent);
+    } else if (schema.getType().equals(Schema.Type.DOUBLE)) {
+      return new FieldDoubleConverter(parent);
+    } else if (schema.getType().equals(Schema.Type.BYTES)) {
+      return new FieldBytesConverter(parent);
+    } else if (schema.getType().equals(Schema.Type.STRING)) {
+      return new FieldStringConverter(parent);
+    } else if (schema.getType().equals(Schema.Type.RECORD)) {
+      return new AvroIndexedRecordConverter(parent, type.asGroupType(), schema);
+    } else if (schema.getType().equals(Schema.Type.ENUM)) {
+      return new FieldEnumConverter(parent,schema);
+    } else if (schema.getType().equals(Schema.Type.ARRAY)) {
+      return new AvroArrayConverter(parent, type, schema);
+    } else if (schema.getType().equals(Schema.Type.MAP)) {
+      return new MapConverter(parent, type, schema);
+    } else if (schema.getType().equals(Schema.Type.UNION)) {
+      return new AvroUnionConverter(parent, type, schema);
+    } else if (schema.getType().equals(Schema.Type.FIXED)) {
+      return new FieldFixedConverter(parent, schema);
+    }
+    throw new UnsupportedOperationException(String.format("Cannot convert Avro type: %s" +
+        " (Parquet type: %s) ", schema, type));
+  }
+
+  private void set(int index, Object value) {
+    this.currentRecord.put(index, value);
+  }
+
+  @Override
+  public Converter getConverter(int fieldIndex) {
+    return converters[fieldIndex];
+  }
+
+  @Override
+  public void start() {
+    // Should do the right thing whether it is generic or specific
+    this.currentRecord = (this.specificClass == null) ?
+        new GenericData.Record(avroSchema) :
+        (IndexedRecord)SpecificData.newInstance(specificClass, avroSchema);
+  }
+
+  @Override
+  public void end() {
+    if (parent != null) {
+      parent.add(currentRecord);
+    }
+  }
+
+  IndexedRecord getCurrentRecord() {
+    return currentRecord;
+  }
+
+  static abstract class ParentValueContainer {
+
+    /**
+     * Adds the value to the parent.
+     */
+    abstract void add(Object value);
+
+  }
+
+  static final class FieldBooleanConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+
+    public FieldBooleanConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBoolean(boolean value) {
+      parent.add(value);
+    }
+
+  }
+
+  static final class FieldIntegerConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+
+    public FieldIntegerConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addInt(int value) {
+      parent.add(value);
+    }
+
+  }
+
+  static final class FieldLongConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+
+    public FieldLongConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addLong(long value) {
+      parent.add(value);
+    }
+
+  }
+
+  static final class FieldFloatConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+
+    public FieldFloatConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addFloat(float value) {
+      parent.add(value);
+    }
+
+  }
+
+  static final class FieldDoubleConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+
+    public FieldDoubleConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addDouble(double value) {
+      parent.add(value);
+    }
+
+  }
+
+  static final class FieldBytesConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+
+    public FieldBytesConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      parent.add(ByteBuffer.wrap(value.getBytes()));
+    }
+
+  }
+
+  static final class FieldStringConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+
+    public FieldStringConverter(ParentValueContainer parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      parent.add(value.toStringUsingUTF8());
+    }
+
+  }
+
+  static final class FieldEnumConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+    private final Class<? extends Enum> enumClass;
+
+    public FieldEnumConverter(ParentValueContainer parent, Schema enumSchema) {
+      this.parent = parent;
+      this.enumClass = SpecificData.get().getClass(enumSchema);
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      Object enumValue = value.toStringUsingUTF8();
+      if (enumClass != null) {
+        enumValue = (Enum.valueOf(enumClass,(String)enumValue));
+      }
+      parent.add(enumValue);
+    }
+  }
+
+  static final class FieldFixedConverter extends PrimitiveConverter {
+
+    private final ParentValueContainer parent;
+    private final Schema avroSchema;
+
+    public FieldFixedConverter(ParentValueContainer parent, Schema avroSchema) {
+      this.parent = parent;
+      this.avroSchema = avroSchema;
+    }
+
+    @Override
+    final public void addBinary(Binary value) {
+      parent.add(new GenericData.Fixed(avroSchema, value.getBytes()));
+    }
+
+  }
+
+  static final class AvroArrayConverter<T> extends GroupConverter {
+
+    private final ParentValueContainer parent;
+    private final Schema avroSchema;
+    private final Converter converter;
+    private GenericArray<T> array;
+
+    public AvroArrayConverter(ParentValueContainer parent, Type parquetSchema,
+        Schema avroSchema) {
+      this.parent = parent;
+      this.avroSchema = avroSchema;
+      Type elementType = parquetSchema.asGroupType().getType(0);
+      Schema elementSchema = avroSchema.getElementType();
+      converter = newConverter(elementSchema, elementType, new ParentValueContainer() {
+        @Override
+        @SuppressWarnings("unchecked")
+        void add(Object value) {
+          array.add((T) value);
+        }
+      });
+    }
+
+    @Override
+    public Converter getConverter(int fieldIndex) {
+      return converter;
+    }
+
+    @Override
+    public void start() {
+      array = new GenericData.Array<T>(0, avroSchema);
+    }
+
+    @Override
+    public void end() {
+      parent.add(array);
+    }
+  }
+
+  static final class AvroUnionConverter<T> extends GroupConverter {
+
+    private final ParentValueContainer parent;
+    private final Converter[] memberConverters;
+    private Object memberValue = null;
+
+    public AvroUnionConverter(ParentValueContainer parent, Type parquetSchema,
+                              Schema avroSchema) {
+      this.parent = parent;
+      GroupType parquetGroup = parquetSchema.asGroupType();
+      this.memberConverters = new Converter[ parquetGroup.getFieldCount()];
+
+      int parquetIndex = 0;
+      for (int index = 0; index < avroSchema.getTypes().size(); index++) {
+        Schema memberSchema = avroSchema.getTypes().get(index);
+        if (!memberSchema.getType().equals(Schema.Type.NULL)) {
+          Type memberType = parquetGroup.getType(parquetIndex);
+          memberConverters[parquetIndex] = newConverter(memberSchema, memberType, new ParentValueContainer() {
+            @Override
+            void add(Object value) {
+              Preconditions.checkArgument(memberValue==null, "Union is resolving to more than one type");
+              memberValue = value;
+            }
+          });
+          parquetIndex++; // Note for nulls the parquetIndex id not increased
+        }
+      }
+    }
+
+    @Override
+    public Converter getConverter(int fieldIndex) {
+      return memberConverters[fieldIndex];
+    }
+
+    @Override
+    public void start() {
+      memberValue = null;
+    }
+
+    @Override
+    public void end() {
+      parent.add(memberValue);
+    }
+  }
+
+  static final class MapConverter<V> extends GroupConverter {
+
+    private final ParentValueContainer parent;
+    private final Converter keyValueConverter;
+    private Map<String, V> map;
+
+    public MapConverter(ParentValueContainer parent, Type parquetSchema,
+        Schema avroSchema) {
+      this.parent = parent;
+      this.keyValueConverter = new MapKeyValueConverter(parquetSchema, avroSchema);
+    }
+
+    @Override
+    public Converter getConverter(int fieldIndex) {
+      return keyValueConverter;
+    }
+
+    @Override
+    public void start() {
+      this.map = new HashMap<String, V>();
+    }
+
+    @Override
+    public void end() {
+      parent.add(map);
+    }
+
+    final class MapKeyValueConverter extends GroupConverter {
+
+      private String key;
+      private V value;
+      private Converter keyConverter;
+      private Converter valueConverter;
+
+      public MapKeyValueConverter(Type parquetSchema, Schema avroSchema) {
+        keyConverter = new PrimitiveConverter() {
+          @Override
+          final public void addBinary(Binary value) {
+            key = value.toStringUsingUTF8();
+          }
+        };
+
+        Type valueType = parquetSchema.asGroupType().getType(0).asGroupType().getType(1);
+        Schema valueSchema = avroSchema.getValueType();
+        valueConverter = newConverter(valueSchema, valueType, new ParentValueContainer() {
+          @Override
+          @SuppressWarnings("unchecked")
+          void add(Object value) {
+            MapKeyValueConverter.this.value = (V) value;
+          }
+        });
+      }
+
+      @Override
+      public Converter getConverter(int fieldIndex) {
+        if (fieldIndex == 0) {
+          return keyConverter;
+        } else if (fieldIndex == 1) {
+          return valueConverter;
+        }
+        throw new IllegalArgumentException("only the key (0) and value (1) fields expected: " + fieldIndex);
+      }
+
+      @Override
+      public void start() {
+        key = null;
+        value = null;
+      }
+
+      @Override
+      public void end() {
+        map.put(key, value);
+      }
+    }
+  }
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesType.java b/parquet-avro/src/main/java/parquet/avro/AvroParquetInputFormat.java
similarity index 60%
copy from parquet-column/src/main/java/parquet/column/values/ValuesType.java
copy to parquet-avro/src/main/java/parquet/avro/AvroParquetInputFormat.java
index 0566303..8c95121 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesType.java
+++ b/parquet-avro/src/main/java/parquet/avro/AvroParquetInputFormat.java
@@ -13,14 +13,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.values;
+package parquet.avro;
+
+import org.apache.avro.generic.IndexedRecord;
+import parquet.avro.AvroReadSupport;
+import parquet.hadoop.ParquetInputFormat;
 
 /**
- * The different type of values we can store in columns
- *
- * @author Julien Le Dem
- *
+ * A Hadoop {@link org.apache.hadoop.mapreduce.InputFormat} for Parquet files.
  */
-public enum ValuesType {
-  REPETITION_LEVEL, DEFINITION_LEVEL, VALUES;
+public class AvroParquetInputFormat extends ParquetInputFormat<IndexedRecord> {
+  public AvroParquetInputFormat() {
+    super(AvroReadSupport.class);
+  }
 }
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroParquetOutputFormat.java b/parquet-avro/src/main/java/parquet/avro/AvroParquetOutputFormat.java
new file mode 100644
index 0000000..67f7ca1
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroParquetOutputFormat.java
@@ -0,0 +1,38 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.mapreduce.Job;
+import parquet.avro.AvroWriteSupport;
+import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.util.ContextUtil;
+
+/**
+ * A Hadoop {@link org.apache.hadoop.mapreduce.OutputFormat} for Parquet files.
+ */
+public class AvroParquetOutputFormat extends ParquetOutputFormat<IndexedRecord> {
+
+  public static void setSchema(Job job, Schema schema) {
+    AvroWriteSupport.setSchema(ContextUtil.getConfiguration(job), schema);
+  }
+
+  public AvroParquetOutputFormat() {
+    super(new AvroWriteSupport());
+  }
+
+}
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroParquetReader.java b/parquet-avro/src/main/java/parquet/avro/AvroParquetReader.java
new file mode 100644
index 0000000..211895c
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroParquetReader.java
@@ -0,0 +1,38 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import java.io.IOException;
+
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.Path;
+import parquet.filter.UnboundRecordFilter;
+import parquet.hadoop.ParquetReader;
+import parquet.hadoop.api.ReadSupport;
+
+/**
+ * Read Avro records from a Parquet file.
+ */
+public class AvroParquetReader<T extends IndexedRecord> extends ParquetReader<T> {
+
+  public AvroParquetReader(Path file) throws IOException {
+    super(file, (ReadSupport<T>) new AvroReadSupport());
+  }
+
+  public AvroParquetReader(Path file, UnboundRecordFilter recordFilter ) throws IOException {
+    super(file, (ReadSupport<T>) new AvroReadSupport(), recordFilter);
+  }
+}
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroParquetWriter.java b/parquet-avro/src/main/java/parquet/avro/AvroParquetWriter.java
new file mode 100644
index 0000000..27d359c
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroParquetWriter.java
@@ -0,0 +1,77 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import java.io.IOException;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.Path;
+import parquet.hadoop.ParquetWriter;
+import parquet.hadoop.api.WriteSupport;
+import parquet.hadoop.metadata.CompressionCodecName;
+
+/**
+ * Write Avro records to a Parquet file.
+ */
+public class AvroParquetWriter<T extends IndexedRecord> extends ParquetWriter<T> {
+
+  /** Create a new {@link AvroParquetWriter}.
+   *
+   * @param file
+   * @param avroSchema
+   * @param compressionCodecName
+   * @param blockSize
+   * @param pageSize
+   * @throws IOException
+   */
+  public AvroParquetWriter(Path file, Schema avroSchema,
+      CompressionCodecName compressionCodecName, int blockSize,
+      int pageSize) throws IOException {
+    super(file, (WriteSupport<T>)new AvroWriteSupport(new AvroSchemaConverter().convert(avroSchema), avroSchema),
+	      compressionCodecName, blockSize, pageSize);
+  }
+
+  /** Create a new {@link AvroParquetWriter}.
+   *
+   * @param file The file name to write to.
+   * @param avroSchema The schema to write with.
+   * @param compressionCodecName Compression code to use, or CompressionCodecName.UNCOMPRESSED
+   * @param blockSize HDFS block size
+   * @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.
+   * @throws IOException
+   */
+  public AvroParquetWriter(Path file, Schema avroSchema,
+                           CompressionCodecName compressionCodecName, int blockSize,
+                           int pageSize, boolean enableDictionary) throws IOException {
+    super(file, (WriteSupport<T>)
+        new AvroWriteSupport(new AvroSchemaConverter().convert(avroSchema),avroSchema),
+        compressionCodecName, blockSize, pageSize, enableDictionary, false);
+  }
+
+  /** Create a new {@link AvroParquetWriter}. The default block size is 50 MB.The default
+   *  page size is 1 MB.  Default compression is no compression. (Inherited from {@link ParquetWriter})
+   *
+   * @param file The file name to write to.
+   * @param avroSchema The schema to write with.
+   * @throws IOException
+   */
+  public AvroParquetWriter(Path file, Schema avroSchema) throws IOException {
+    this(file, avroSchema, CompressionCodecName.UNCOMPRESSED,
+	  DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
+  }
+
+}
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroReadSupport.java b/parquet-avro/src/main/java/parquet/avro/AvroReadSupport.java
new file mode 100644
index 0000000..1ea8929
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroReadSupport.java
@@ -0,0 +1,43 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import java.util.Map;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import parquet.hadoop.api.ReadSupport;
+import parquet.io.api.RecordMaterializer;
+import parquet.schema.MessageType;
+
+/**
+ * Avro implementation of {@link ReadSupport} for Avro {@link IndexedRecord}s which cover both Avro Specific and
+ * Generic. Users should use {@link AvroParquetReader} or {@link AvroParquetInputFormat} rather than using
+ * this class directly.
+ */
+public class AvroReadSupport extends ReadSupport<IndexedRecord> {
+
+  @Override
+  public ReadContext init(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema) {
+    return new ReadContext(fileSchema);
+  }
+
+  @Override
+  public RecordMaterializer<IndexedRecord> prepareForRead(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema, ReadContext readContext) {
+    Schema avroSchema = new Schema.Parser().parse(keyValueMetaData.get("avro.schema"));
+    return new AvroRecordMaterializer(readContext.getRequestedSchema(), avroSchema);
+  }
+}
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroRecordMaterializer.java b/parquet-avro/src/main/java/parquet/avro/AvroRecordMaterializer.java
new file mode 100644
index 0000000..e9df29f
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroRecordMaterializer.java
@@ -0,0 +1,41 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import parquet.io.api.GroupConverter;
+import parquet.io.api.RecordMaterializer;
+import parquet.schema.MessageType;
+
+class AvroRecordMaterializer extends RecordMaterializer<IndexedRecord> {
+
+  private AvroIndexedRecordConverter root;
+
+  public AvroRecordMaterializer(MessageType requestedSchema, Schema avroSchema) {
+    this.root = new AvroIndexedRecordConverter(requestedSchema, avroSchema);
+  }
+
+  @Override
+  public IndexedRecord getCurrentRecord() {
+    return root.getCurrentRecord();
+  }
+
+  @Override
+  public GroupConverter getRootConverter() {
+    return root;
+  }
+}
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java b/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java
new file mode 100644
index 0000000..9a5524a
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java
@@ -0,0 +1,164 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import java.util.*;
+
+import org.apache.avro.Schema;
+import parquet.schema.GroupType;
+import parquet.schema.MessageType;
+import parquet.schema.OriginalType;
+import parquet.schema.PrimitiveType;
+import parquet.schema.Type;
+
+import static parquet.schema.OriginalType.*;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.*;
+
+/**
+ * <p>
+ * Converts an Avro schema into a Parquet schema. See package documentation for details
+ * of the mapping.
+ * </p>
+ */
+public class AvroSchemaConverter {
+
+  /**
+   * Given a schema, check to see if it is a union of a null type and a regular schema,
+   * and then return the non-null sub-schema. Otherwise, return the given schema.
+   * 
+   * @param schema The schema to check
+   * @return The non-null portion of a union schema, or the given schema
+   */
+  public static Schema getNonNull(Schema schema) {
+    if (schema.getType().equals(Schema.Type.UNION)) {
+      List<Schema> schemas = schema.getTypes();
+      if (schemas.size() == 2) {
+        if (schemas.get(0).getType().equals(Schema.Type.NULL)) {
+          return schemas.get(1);
+        } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) {
+          return schemas.get(0);
+        } else {
+          return schema;
+        }
+      } else {
+        return schema;
+      }
+    } else {
+      return schema;
+    }
+  }
+  
+  public MessageType convert(Schema avroSchema) {
+    if (!avroSchema.getType().equals(Schema.Type.RECORD)) {
+      throw new IllegalArgumentException("Avro schema must be a record.");
+    }
+    return new MessageType(avroSchema.getFullName(), convertFields(avroSchema.getFields()));
+  }
+
+  private List<Type> convertFields(List<Schema.Field> fields) {
+    List<Type> types = new ArrayList<Type>();
+    for (Schema.Field field : fields) {
+      if (field.schema().getType().equals(Schema.Type.NULL)) {
+        continue; // Avro nulls are not encoded, unless they are null unions
+      }
+      types.add(convertField(field));
+    }
+    return types;
+  }
+
+  private Type convertField(String fieldName, Schema schema) {
+    return convertField(fieldName, schema, Type.Repetition.REQUIRED);
+  }
+
+  private Type convertField(String fieldName, Schema schema, Type.Repetition repetition) {
+    Schema.Type type = schema.getType();
+    if (type.equals(Schema.Type.BOOLEAN)) {
+      return primitive(fieldName, BOOLEAN, repetition);
+    } else if (type.equals(Schema.Type.INT)) {
+      return primitive(fieldName, INT32, repetition);
+    } else if (type.equals(Schema.Type.LONG)) {
+      return primitive(fieldName, INT64, repetition);
+    } else if (type.equals(Schema.Type.FLOAT)) {
+      return primitive(fieldName, FLOAT, repetition);
+    } else if (type.equals(Schema.Type.DOUBLE)) {
+      return primitive(fieldName, DOUBLE, repetition);
+    } else if (type.equals(Schema.Type.BYTES)) {
+      return primitive(fieldName, BINARY, repetition);
+    } else if (type.equals(Schema.Type.STRING)) {
+      return primitive(fieldName, BINARY, repetition, UTF8);
+    } else if (type.equals(Schema.Type.RECORD)) {
+      return new GroupType(repetition, fieldName, convertFields(schema.getFields()));
+    } else if (type.equals(Schema.Type.ENUM)) {
+      return primitive(fieldName, BINARY, repetition, ENUM);
+    } else if (type.equals(Schema.Type.ARRAY)) {
+      return new GroupType(repetition, fieldName, LIST,
+          convertField("array", schema.getElementType(), Type.Repetition.REPEATED));
+    } else if (type.equals(Schema.Type.MAP)) {
+      Type keyType = convertField("key", Schema.create(Schema.Type.STRING));
+      Type valType = convertField("value", schema.getValueType());
+      return new GroupType(repetition, fieldName, MAP,
+          new GroupType(Type.Repetition.REPEATED, "map", MAP_KEY_VALUE, keyType, valType));
+    } else if (type.equals(Schema.Type.FIXED)) {
+      return primitive(fieldName, FIXED_LEN_BYTE_ARRAY, repetition);
+    } else if (type.equals(Schema.Type.UNION)) {
+      return convertUnion(fieldName, schema, repetition);
+    }
+    throw new UnsupportedOperationException("Cannot convert Avro type " + type);
+  }
+
+  private Type convertUnion(String fieldName, Schema schema, Type.Repetition repetition) {
+    List<Schema> nonNullSchemas = new ArrayList(schema.getTypes().size());
+    for (Schema childSchema : schema.getTypes()) {
+      if (childSchema.getType().equals(Schema.Type.NULL)) {
+        repetition = Type.Repetition.OPTIONAL;
+      } else {
+        nonNullSchemas.add(childSchema);
+      }
+    }
+    // If we only get a null and one other type then its a simple optional field
+    // otherwise construct a union container
+    switch (nonNullSchemas.size()) {
+      case 0:
+        throw new UnsupportedOperationException("Cannot convert Avro union of only nulls");
+
+      case 1:
+        return convertField(fieldName, nonNullSchemas.get(0), Type.Repetition.OPTIONAL); // Simple optional field
+
+      default: // complex union type
+        List<Type> unionTypes = new ArrayList(nonNullSchemas.size());
+        int index = 0;
+        for (Schema childSchema : nonNullSchemas) {
+          unionTypes.add( convertField("member" + index++, childSchema, Type.Repetition.OPTIONAL));
+        }
+        return new GroupType(repetition, fieldName, unionTypes);
+    }
+  }
+
+  private Type convertField(Schema.Field field) {
+    return convertField(field.name(), field.schema());
+  }
+
+  private Type primitive(String name, PrimitiveType.PrimitiveTypeName primitive,
+      Type.Repetition repetition, OriginalType originalType) {
+    return new PrimitiveType(repetition, primitive, name, originalType);
+  }
+
+  private PrimitiveType primitive(String name, PrimitiveType.PrimitiveTypeName
+      primitive, Type.Repetition repetition) {
+    return new PrimitiveType(repetition, primitive, name, null);
+  }
+
+}
\ No newline at end of file
diff --git a/parquet-avro/src/main/java/parquet/avro/AvroWriteSupport.java b/parquet-avro/src/main/java/parquet/avro/AvroWriteSupport.java
new file mode 100644
index 0000000..fc5ec70
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/AvroWriteSupport.java
@@ -0,0 +1,215 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericArray;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.specific.SpecificData;
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.avro.util.Utf8;
+import org.apache.hadoop.conf.Configuration;
+import parquet.hadoop.api.WriteSupport;
+import parquet.io.api.Binary;
+import parquet.io.api.RecordConsumer;
+import parquet.schema.GroupType;
+import parquet.schema.MessageType;
+import parquet.schema.Type;
+
+/**
+ * Avro implementation of {@link WriteSupport} for {@link IndexedRecord}s - both Avro Generic and Specific.
+ * Users should use {@link AvroParquetWriter} or {@link AvroParquetOutputFormat} rather than using
+ * this class directly.
+ */
+public class AvroWriteSupport extends WriteSupport<IndexedRecord> {
+
+  private RecordConsumer recordConsumer;
+  private MessageType rootSchema;
+  private Schema rootAvroSchema;
+
+  public AvroWriteSupport() {
+  }
+
+  public AvroWriteSupport(MessageType schema, Schema avroSchema) {
+    this.rootSchema = schema;
+    this.rootAvroSchema = avroSchema;
+  }
+
+  public static void setSchema(Configuration configuration, Schema schema) {
+    configuration.set("parquet.avro.schema", schema.toString());
+  }
+
+  @Override
+  public WriteContext init(Configuration configuration) {
+    if (rootAvroSchema == null) {
+      rootAvroSchema = new Schema.Parser().parse(configuration.get("parquet.avro.schema"));
+      rootSchema = new AvroSchemaConverter().convert(rootAvroSchema);
+    }
+    Map<String, String> extraMetaData = new HashMap<String, String>();
+    extraMetaData.put("avro.schema", rootAvroSchema.toString());
+    return new WriteContext(rootSchema, extraMetaData);
+  }
+
+  @Override
+  public void prepareForWrite(RecordConsumer recordConsumer) {
+    this.recordConsumer = recordConsumer;
+  }
+
+  @Override
+  public void write(IndexedRecord record) {
+    recordConsumer.startMessage();
+    writeRecordFields(rootSchema, rootAvroSchema, record);
+    recordConsumer.endMessage();
+  }
+
+  private void writeRecord(GroupType schema, Schema avroSchema,
+                           IndexedRecord record) {
+    recordConsumer.startGroup();
+    writeRecordFields(schema, avroSchema, record);
+    recordConsumer.endGroup();
+  }
+
+  private void writeRecordFields(GroupType schema, Schema avroSchema,
+                                 IndexedRecord record) {
+    List<Type> fields = schema.getFields();
+    List<Schema.Field> avroFields = avroSchema.getFields();
+    int index = 0; // parquet ignores Avro nulls, so index may differ
+    for (int avroIndex = 0; avroIndex < avroFields.size(); avroIndex++) {
+      Schema.Field avroField = avroFields.get(avroIndex);
+      if (avroField.schema().getType().equals(Schema.Type.NULL)) {
+        continue;
+      }
+      Type fieldType = fields.get(index);
+      Object value = record.get(avroIndex);
+      if (value != null) {
+        recordConsumer.startField(fieldType.getName(), index);
+        writeValue(fieldType, avroField.schema(), value);
+        recordConsumer.endField(fieldType.getName(), index);
+      } else if (fieldType.getRepetition() == Type.Repetition.REQUIRED) {
+        throw new RuntimeException("Null-value for required field: " + avroField.name());
+      }
+      index++;
+    }
+  }
+  
+  private <T> void writeArray(GroupType schema, Schema avroSchema,
+      Iterable<T> array) {
+    recordConsumer.startGroup(); // group wrapper (original type LIST)
+    recordConsumer.startField("array", 0);
+    for (T elt : array) {
+      writeValue(schema.getType(0), avroSchema.getElementType(), elt);
+    }
+    recordConsumer.endField("array", 0);
+    recordConsumer.endGroup();
+  }
+
+  private <V> void writeMap(GroupType schema, Schema avroSchema, Map<String, V> map) {
+    GroupType innerGroup = schema.getType(0).asGroupType();
+    Type keyType = innerGroup.getType(0);
+    Type valueType = innerGroup.getType(1);
+    Schema keySchema = Schema.create(Schema.Type.STRING);
+
+    recordConsumer.startGroup(); // group wrapper (original type MAP)
+    recordConsumer.startField("map", 0);
+    recordConsumer.startGroup(); // "repeated" group wrapper
+    recordConsumer.startField("key", 0);
+    for (String key : map.keySet()) {
+      writeValue(keyType, keySchema, key);
+    }
+    recordConsumer.endField("key", 0);
+    recordConsumer.startField("value", 1);
+    for (V value : map.values()) {
+      writeValue(valueType, avroSchema.getValueType(), value);
+    }
+    recordConsumer.endField("value", 1);
+    recordConsumer.endGroup();
+    recordConsumer.endField("map", 0);
+    recordConsumer.endGroup();
+  }
+
+  private void writeUnion(GroupType parquetSchema, Schema avroSchema, Object value) {
+
+    recordConsumer.startGroup();
+
+    // ResolveUnion will tell us which of the union member types to deserialise
+    int avroIndex = GenericData.get().resolveUnion(avroSchema, value);
+
+    // For parquet's schema we skip nulls
+    GroupType parquetGroup = parquetSchema.asGroupType();
+    int parquetIndex = avroIndex;
+    for ( int i=0; i<avroIndex; i++) {
+      if ( avroSchema.getTypes().get(i).getType().equals( Schema.Type.NULL )) {
+        parquetIndex--;
+      }
+    }
+
+    // Sparsely populated method of encoding unions, each member has its own set of columns
+    String memberName = "member" + parquetIndex;
+    recordConsumer.startField(memberName, parquetIndex);
+    writeValue(parquetGroup.getType(parquetIndex), avroSchema.getTypes().get(avroIndex), value);
+    recordConsumer.endField(memberName, parquetIndex);
+
+    recordConsumer.endGroup();
+  }
+
+  @SuppressWarnings("unchecked")
+  private void writeValue(Type type, Schema avroSchema, Object value) {
+    Schema nonNullAvroSchema =  AvroSchemaConverter.getNonNull(avroSchema);
+    Schema.Type avroType = nonNullAvroSchema.getType();
+    if (avroType.equals(Schema.Type.BOOLEAN)) {
+      recordConsumer.addBoolean((Boolean) value);
+    } else if (avroType.equals(Schema.Type.INT)) {
+      recordConsumer.addInteger(((Number) value).intValue());
+    } else if (avroType.equals(Schema.Type.LONG)) {
+      recordConsumer.addLong(((Number) value).longValue());
+    } else if (avroType.equals(Schema.Type.FLOAT)) {
+      recordConsumer.addFloat(((Number) value).floatValue());
+    } else if (avroType.equals(Schema.Type.DOUBLE)) {
+      recordConsumer.addDouble(((Number) value).doubleValue());
+    } else if (avroType.equals(Schema.Type.BYTES)) {
+      recordConsumer.addBinary(Binary.fromByteBuffer((ByteBuffer) value));
+    } else if (avroType.equals(Schema.Type.STRING)) {
+      recordConsumer.addBinary(fromAvroString(value));
+    } else if (avroType.equals(Schema.Type.RECORD)) {
+      writeRecord((GroupType) type, nonNullAvroSchema, (IndexedRecord) value);
+    } else if (avroType.equals(Schema.Type.ENUM)) {
+      recordConsumer.addBinary(Binary.fromString(value.toString()));
+    } else if (avroType.equals(Schema.Type.ARRAY)) {
+      writeArray((GroupType) type, nonNullAvroSchema, (Iterable<?>) value);
+    } else if (avroType.equals(Schema.Type.MAP)) {
+      writeMap((GroupType) type, nonNullAvroSchema, (Map<String, ?>) value);
+    } else if (avroType.equals(Schema.Type.UNION)) {
+      writeUnion((GroupType) type, nonNullAvroSchema, value);
+    } else if (avroType.equals(Schema.Type.FIXED)) {
+      recordConsumer.addBinary(Binary.fromByteArray(((GenericFixed) value).bytes()));
+    }
+  }
+
+  private Binary fromAvroString(Object value) {
+    if (value instanceof Utf8) {
+      Utf8 utf8 = (Utf8) value;
+      return Binary.fromByteArray(utf8.getBytes(), 0, utf8.getByteLength());
+    }
+    return Binary.fromString(value.toString());
+  }
+
+}
diff --git a/parquet-avro/src/main/java/parquet/avro/package-info.java b/parquet-avro/src/main/java/parquet/avro/package-info.java
new file mode 100644
index 0000000..c1553e4
--- /dev/null
+++ b/parquet-avro/src/main/java/parquet/avro/package-info.java
@@ -0,0 +1,89 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ */
+/**
+ *
+ * <p>
+ * Provides classes to store Avro data in Parquet files. Avro schemas are converted to
+ * parquet schemas as follows. Only record schemas are converted,
+ * other top-level schema types are not converted and attempting to do so will result
+ * in an error. Avro types are converted to Parquet types using the mapping shown here:
+ * </p>
+ *
+ * <table>
+ *   <tr>
+ *     <th>Avro type</th>
+ *     <th>Parquet type</th>
+ *   </tr>
+ *   <tr>
+ *     <td>null</td>
+ *     <td>no type (the field is not encoded in Parquet), unless a null union</td>
+ *   </tr>
+ *   <tr>
+ *     <td>boolean</td>
+ *     <td>boolean</td>
+ *   </tr>
+ *   <tr>
+ *     <td>int</td>
+ *     <td>int32</td>
+ *   </tr>
+ *   <tr>
+ *     <td>long</td>
+ *     <td>int64</td>
+ *   </tr>
+ *   <tr>
+ *     <td>float</td>
+ *     <td>float</td>
+ *   </tr>
+ *   <tr>
+ *     <td>double</td>
+ *     <td>double</td>
+ *   </tr>
+ *   <tr>
+ *     <td>bytes</td>
+ *     <td>binary</td>
+ *   </tr>
+ *   <tr>
+ *     <td>string</td>
+ *     <td>binary (with original type UTF8)</td>
+ *   </tr>
+ *   <tr>
+ *     <td>record</td>
+ *     <td>group containing nested fields</td>
+ *   </tr>
+ *   <tr>
+ *     <td>enum</td>
+ *     <td>binary (with original type ENUM)</td>
+ *   </tr>
+ *   <tr>
+ *     <td>array</td>
+ *     <td>group (with original type LIST) containing one repeated group field</td>
+ *   </tr>
+ *   <tr>
+ *     <td>map</td>
+ *     <td>group (with original type MAP) containing one repeated group
+ *     field (with original type MAP_KEY_VALUE) of (key, value)</td>
+ *   </tr>
+ *   <tr>
+ *     <td>fixed</td>
+ *     <td>fixed_len_byte_array</td>
+ *   </tr>
+ *   <tr>
+ *     <td>union</td>
+ *     <td>an optional type, in the case of a null union, otherwise not supported</td>
+ *   </tr>
+ * </table>
+ */
+package parquet.avro;
diff --git a/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java b/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java
new file mode 100644
index 0000000..2b5ce9a
--- /dev/null
+++ b/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java
@@ -0,0 +1,113 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import com.google.common.io.Resources;
+import java.util.Arrays;
+import org.apache.avro.Schema;
+import org.codehaus.jackson.node.NullNode;
+import org.junit.Test;
+import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestAvroSchemaConverter {
+
+  private void testConversion(Schema avroSchema, String schemaString) throws
+      Exception {
+    AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter();
+    MessageType schema = avroSchemaConverter.convert(avroSchema);
+    MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString);
+    assertEquals("converting " + schema + " to " + schemaString, expectedMT.toString(),
+        schema.toString());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testTopLevelMustBeARecord() {
+    new AvroSchemaConverter().convert(Schema.create(Schema.Type.INT));
+  }
+
+  @Test
+  public void testAllTypes() throws Exception {
+    Schema schema = new Schema.Parser().parse(
+        Resources.getResource("all.avsc").openStream());
+    testConversion(
+        schema,
+        "message parquet.avro.myrecord {\n" +
+        // Avro nulls are not encoded, unless they are null unions
+        "  required boolean myboolean;\n" +
+        "  required int32 myint;\n" +
+        "  required int64 mylong;\n" +
+        "  required float myfloat;\n" +
+        "  required double mydouble;\n" +
+        "  required binary mybytes;\n" +
+        "  required binary mystring (UTF8);\n" +
+        "  required group mynestedrecord {\n" +
+        "    required int32 mynestedint;\n" +
+        "  }\n" +
+        "  required binary myenum (ENUM);\n" +
+        "  required group myarray (LIST) {\n" +
+        "    repeated int32 array;\n" +
+        "  }\n" +
+        "  required group mymap (MAP) {\n" +
+        "    repeated group map (MAP_KEY_VALUE) {\n" +
+        "      required binary key (UTF8);\n" +
+        "      required int32 value;\n" +
+        "    }\n" +
+        "  }\n" +
+        "  required fixed_len_byte_array myfixed;\n" +
+        "}\n");
+  }
+
+  @Test
+  public void testOptionalFields() throws Exception {
+    Schema schema = Schema.createRecord("record1", null, null, false);
+    Schema optionalInt = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type
+        .NULL),
+        Schema.create(Schema.Type.INT)));
+    schema.setFields(Arrays.asList(
+        new Schema.Field("myint", optionalInt, null, NullNode.getInstance())
+    ));
+    testConversion(
+        schema,
+        "message record1 {\n" +
+        "  optional int32 myint;\n" +
+        "}\n");
+  }
+
+  @Test
+  public void testUnionOfTwoTypes() throws Exception {
+    Schema schema = Schema.createRecord("record2", null, null, false);
+    Schema multipleTypes = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type
+        .NULL),
+        Schema.create(Schema.Type.INT),
+        Schema.create(Schema.Type.FLOAT)));
+    schema.setFields(Arrays.asList(
+        new Schema.Field("myunion", multipleTypes, null, NullNode.getInstance())
+    ));
+
+    // Avro union is modelled using optional data members of thw different types;
+    testConversion(
+        schema,
+        "message record2 {\n" +
+            "  optional group myunion {\n" +
+            "    optional int32 member0;\n" +
+            "    optional float member1;\n" +
+            "  }\n" +
+            "}\n");
+  }
+}
diff --git a/parquet-avro/src/test/java/parquet/avro/TestInputOutputFormat.java b/parquet-avro/src/test/java/parquet/avro/TestInputOutputFormat.java
new file mode 100644
index 0000000..e4cf809
--- /dev/null
+++ b/parquet-avro/src/test/java/parquet/avro/TestInputOutputFormat.java
@@ -0,0 +1,141 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Arrays;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.junit.Test;
+import parquet.Log;
+
+import static java.lang.Thread.sleep;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestInputOutputFormat {
+  private static final Log LOG = Log.getLog(TestInputOutputFormat.class);
+
+  private static Schema avroSchema;
+  static {
+    avroSchema = Schema.createRecord("record1", null, null, false);
+    avroSchema.setFields(
+        Arrays.asList(new Schema.Field("a",
+            Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.INT), Schema.create(Schema.Type.NULL))),
+            null, null)));
+  }
+
+  public static GenericRecord nextRecord(Integer i) {
+    return new GenericRecordBuilder(avroSchema).set("a", i).build();
+  };
+
+  public static class MyMapper extends Mapper<LongWritable, Text, Void, GenericRecord> {
+
+    public void run(Context context) throws IOException ,InterruptedException {
+      for (int i = 0; i < 10; i++) {
+        GenericRecord a;
+        a = TestInputOutputFormat.nextRecord(i == 4 ? null : i);
+        context.write(null, a);
+      }
+    }
+  }
+
+  public static class MyMapper2 extends Mapper<Void, GenericRecord, LongWritable, Text> {
+    protected void map(Void key, GenericRecord value, Context context) throws IOException ,InterruptedException {
+      context.write(null, new Text(value.toString()));
+    }
+
+  }
+
+  @Test
+  public void testReadWrite() throws Exception {
+
+    final Configuration conf = new Configuration();
+    final Path inputPath = new Path("src/test/java/parquet/avro/TestInputOutputFormat.java");
+    final Path parquetPath = new Path("target/test/hadoop/TestInputOutputFormat/parquet");
+    final Path outputPath = new Path("target/test/hadoop/TestInputOutputFormat/out");
+    final FileSystem fileSystem = parquetPath.getFileSystem(conf);
+    fileSystem.delete(parquetPath, true);
+    fileSystem.delete(outputPath, true);
+    {
+      final Job job = new Job(conf, "write");
+
+      // input not really used
+      TextInputFormat.addInputPath(job, inputPath);
+      job.setInputFormatClass(TextInputFormat.class);
+
+      job.setMapperClass(TestInputOutputFormat.MyMapper.class);
+      job.setNumReduceTasks(0);
+
+      job.setOutputFormatClass(AvroParquetOutputFormat.class);
+      AvroParquetOutputFormat.setOutputPath(job, parquetPath);
+      AvroParquetOutputFormat.setSchema(job, avroSchema);
+
+      waitForJob(job);
+    }
+    {
+      final Job job = new Job(conf, "read");
+      job.setInputFormatClass(AvroParquetInputFormat.class);
+      AvroParquetInputFormat.setInputPaths(job, parquetPath);
+
+      job.setMapperClass(TestInputOutputFormat.MyMapper2.class);
+      job.setNumReduceTasks(0);
+
+      job.setOutputFormatClass(TextOutputFormat.class);
+      TextOutputFormat.setOutputPath(job, outputPath);
+
+      waitForJob(job);
+    }
+
+    final BufferedReader out = new BufferedReader(new FileReader(new File(outputPath.toString(), "part-m-00000")));
+    String lineOut = null;
+    int lineNumber = 0;
+    while ((lineOut = out.readLine()) != null) {
+      lineOut = lineOut.substring(lineOut.indexOf("\t") + 1);
+      GenericRecord a = nextRecord(lineNumber == 4 ? null : lineNumber);
+      assertEquals("line " + lineNumber, a.toString(), lineOut);
+      ++ lineNumber;
+    }
+    assertNull("line " + lineNumber, out.readLine());
+    out.close();
+  }
+
+  private void waitForJob(Job job) throws Exception {
+    job.submit();
+    while (!job.isComplete()) {
+      LOG.debug("waiting for job " + job.getJobName());
+      sleep(100);
+    }
+    LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE"));
+    if (!job.isSuccessful()) {
+      throw new RuntimeException("job failed " + job.getJobName());
+    }
+  }
+
+}
diff --git a/parquet-avro/src/test/java/parquet/avro/TestReadWrite.java b/parquet-avro/src/test/java/parquet/avro/TestReadWrite.java
new file mode 100644
index 0000000..a5e6e9b
--- /dev/null
+++ b/parquet-avro/src/test/java/parquet/avro/TestReadWrite.java
@@ -0,0 +1,99 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Resources;
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestReadWrite {
+
+  @Test
+  public void test() throws Exception {
+    Schema schema = new Schema.Parser().parse(
+        Resources.getResource("all-minus-fixed.avsc").openStream());
+
+    File tmp = File.createTempFile(getClass().getSimpleName(), ".tmp");
+    tmp.deleteOnExit();
+    tmp.delete();
+    Path file = new Path(tmp.getPath());
+
+    AvroParquetWriter<GenericRecord> writer = new
+        AvroParquetWriter<GenericRecord>(file, schema);
+
+    GenericData.Record nestedRecord = new GenericRecordBuilder(
+          schema.getField("mynestedrecord").schema())
+        .set("mynestedint", 1).build();
+
+    List<Integer> integerArray = Arrays.asList(1, 2, 3);
+    GenericData.Array<Integer> genericIntegerArray = new GenericData.Array<Integer>(
+                Schema.createArray(Schema.create(Schema.Type.INT)), integerArray);
+
+    GenericData.Record record = new GenericRecordBuilder(schema)
+        .set("mynull", null)
+        .set("myboolean", true)
+        .set("myint", 1)
+        .set("mylong", 2L)
+        .set("myfloat", 3.1f)
+        .set("mydouble", 4.1)
+        .set("mybytes", ByteBuffer.wrap("hello".getBytes(Charsets.UTF_8)))
+        .set("mystring", "hello")
+        .set("mynestedrecord", nestedRecord)
+        .set("myenum", "a")
+        .set("myarray", genericIntegerArray)
+        .set("myoptionalarray", genericIntegerArray)
+        .set("mymap", ImmutableMap.of("a", 1, "b", 2))
+        // TODO: support fixed encoding by plumbing in FIXED_LEN_BYTE_ARRAY
+        //.set("myfixed", new GenericData.Fixed(Schema.createFixed("ignored", null, null, 1),
+        //    new byte[] { (byte) 65 }))
+        .build();
+    writer.write(record);
+    writer.close();
+
+    AvroParquetReader<GenericRecord> reader = new AvroParquetReader<GenericRecord>(file);
+    GenericRecord nextRecord = reader.read();
+
+    assertNotNull(nextRecord);
+    assertEquals(null, nextRecord.get("mynull"));
+    assertEquals(true, nextRecord.get("myboolean"));
+    assertEquals(1, nextRecord.get("myint"));
+    assertEquals(2L, nextRecord.get("mylong"));
+    assertEquals(3.1f, nextRecord.get("myfloat"));
+    assertEquals(4.1, nextRecord.get("mydouble"));
+    assertEquals(ByteBuffer.wrap("hello".getBytes(Charsets.UTF_8)), nextRecord.get("mybytes"));
+    assertEquals("hello", nextRecord.get("mystring"));
+    assertEquals("a", nextRecord.get("myenum"));
+    assertEquals(nestedRecord, nextRecord.get("mynestedrecord"));
+    assertEquals(integerArray, nextRecord.get("myarray"));
+    assertEquals(integerArray, nextRecord.get("myoptionalarray"));
+    assertEquals(ImmutableMap.of("a", 1, "b", 2), nextRecord.get("mymap"));
+    //assertEquals(new byte[] { (byte) 65 }, nextRecord.get("myfixed"));
+  }
+}
diff --git a/parquet-avro/src/test/java/parquet/avro/TestSpecificReadWrite.java b/parquet-avro/src/test/java/parquet/avro/TestSpecificReadWrite.java
new file mode 100644
index 0000000..aaa5d9b
--- /dev/null
+++ b/parquet-avro/src/test/java/parquet/avro/TestSpecificReadWrite.java
@@ -0,0 +1,180 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.avro;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Test;
+import parquet.hadoop.ParquetReader;
+import parquet.hadoop.ParquetWriter;
+import parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static parquet.filter.ColumnRecordFilter.column;
+import static parquet.filter.ColumnPredicates.equalTo;
+import static parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static parquet.hadoop.ParquetWriter.DEFAULT_PAGE_SIZE;
+
+/**
+ * Other tests exercise the use of Avro Generic, a dynamic data representation. This class focuses
+ * on Avro Speific whose schemas are pre-compiled to POJOs with built in SerDe for faster serialization.
+ */
+public class TestSpecificReadWrite {
+
+  @Test
+  public void testReadWriteSpecific() throws IOException {
+    Path path = writeCarsToParquetFile(10, CompressionCodecName.UNCOMPRESSED, false);
+    ParquetReader<Car> reader = new AvroParquetReader<Car>(path);
+    for (int i = 0; i < 10; i++) {
+      assertEquals(getVwPolo().toString(),reader.read().toString());
+      assertEquals(getVwPassat().toString(),reader.read().toString());
+      assertEquals(getBmwMini().toString(),reader.read().toString());
+    }
+    assertNull(reader.read());
+  }
+
+  @Test
+  public void testReadWriteSpecificWithDictionary() throws IOException {
+    Path path = writeCarsToParquetFile(10, CompressionCodecName.UNCOMPRESSED, true);
+    ParquetReader<Car> reader = new AvroParquetReader<Car>(path);
+    for (int i = 0; i < 10; i++) {
+      assertEquals(getVwPolo().toString(),reader.read().toString());
+      assertEquals(getVwPassat().toString(),reader.read().toString());
+      assertEquals(getBmwMini().toString(),reader.read().toString());
+    }
+    assertNull(reader.read());
+  }
+
+  @Test
+  public void testFilterMatchesMultiple() throws IOException {
+
+    Path path = writeCarsToParquetFile(10, CompressionCodecName.UNCOMPRESSED, false);
+
+    ParquetReader<Car> reader = new AvroParquetReader<Car>(path, column("make", equalTo("Volkswagen")));
+    for (int i = 0; i < 10; i++) {
+      assertEquals(getVwPolo().toString(),reader.read().toString());
+      assertEquals(getVwPassat().toString(),reader.read().toString());
+    }
+    assertNull( reader.read());
+  }
+
+  @Test
+  public void testFilterWithDictionary() throws IOException {
+
+    Path path = writeCarsToParquetFile(1,CompressionCodecName.UNCOMPRESSED,true);
+
+    ParquetReader<Car> reader = new AvroParquetReader<Car>(path, column("make", equalTo("Volkswagen")));
+    assertEquals(getVwPolo().toString(),reader.read().toString());
+    assertEquals(getVwPassat().toString(),reader.read().toString());
+    assertNull( reader.read());
+  }
+
+  @Test
+  public void testFilterOnSubAttribute() throws IOException {
+
+    Path path = writeCarsToParquetFile(1, CompressionCodecName.UNCOMPRESSED, false);
+
+    ParquetReader<Car> reader = new AvroParquetReader<Car>(path, column("engine.type", equalTo(EngineType.DIESEL)));
+    assertEquals(reader.read().toString(),getVwPassat().toString());
+    assertNull( reader.read());
+
+    reader = new AvroParquetReader<Car>(path, column("engine.capacity", equalTo(1.4f)));
+    assertEquals(getVwPolo().toString(),reader.read().toString());
+    assertNull( reader.read());
+
+
+    reader = new AvroParquetReader<Car>(path, column("engine.hasTurboCharger", equalTo(true)));
+    assertEquals(getBmwMini().toString(),reader.read().toString());
+    assertNull( reader.read());
+  }
+
+  private Path writeCarsToParquetFile( int num, CompressionCodecName compression, boolean enableDictionary) throws IOException {
+    File tmp = File.createTempFile(getClass().getSimpleName(), ".tmp");
+    tmp.deleteOnExit();
+    tmp.delete();
+    Path path = new Path(tmp.getPath());
+
+    Car vwPolo   = getVwPolo();
+    Car vwPassat = getVwPassat();
+    Car bmwMini  = getBmwMini();
+
+    ParquetWriter<Car> writer = new AvroParquetWriter<Car>(path,Car.SCHEMA$, compression,
+        DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE, enableDictionary);
+    for (int i = 0; i < num; i++) {
+      writer.write(vwPolo);
+      writer.write(vwPassat);
+      writer.write(bmwMini);
+    }
+    writer.close();
+    return path;
+  }
+
+  public static Car getVwPolo() {
+    return Car.newBuilder()
+        .setYear(2010)
+        .setRegistration("A123 GTR")
+        .setMake("Volkswagen")
+        .setModel("Polo")
+        .setDoors(4)
+        .setEngine(Engine.newBuilder().setType(EngineType.PETROL)
+                  .setCapacity(1.4f).setHasTurboCharger(false).build())
+        .setOptionalExtra(
+            Stereo.newBuilder().setMake("Blaupunkt").setSpeakers(4).build())
+        .setServiceHistory(ImmutableList.of(
+            Service.newBuilder().setDate(1325376000l).setMechanic("Jim").build(),
+            Service.newBuilder().setDate(1356998400l).setMechanic("Mike").build()
+            ))
+        .build();
+  }
+
+  public static Car getVwPassat() {
+    return Car.newBuilder()
+        .setYear(2010)
+        .setRegistration("A123 GXR")
+        .setMake("Volkswagen")
+        .setModel("Passat")
+        .setDoors(5)
+        .setEngine(Engine.newBuilder().setType(EngineType.DIESEL)
+            .setCapacity(2.0f).setHasTurboCharger(false).build())
+        .setOptionalExtra(
+            LeatherTrim.newBuilder().setColour("Black").build())
+        .setServiceHistory(ImmutableList.of(
+            Service.newBuilder().setDate(1325376000l).setMechanic("Jim").build()
+        ))
+        .build();
+  }
+
+  public static Car getBmwMini() {
+    return Car.newBuilder()
+        .setYear(2010)
+        .setRegistration("A124 GSR")
+        .setMake("BMW")
+        .setModel("Mini")
+        .setDoors(4)
+        .setEngine(Engine.newBuilder().setType(EngineType.PETROL)
+            .setCapacity(1.6f).setHasTurboCharger(true).build())
+        .setOptionalExtra(null)
+        .setServiceHistory(ImmutableList.of(
+            Service.newBuilder().setDate(1356998400l).setMechanic("Mike").build()
+        ))
+        .build();
+  }
+}
diff --git a/parquet-avro/src/test/resources/all-minus-fixed.avsc b/parquet-avro/src/test/resources/all-minus-fixed.avsc
new file mode 100644
index 0000000..9c0fd4e
--- /dev/null
+++ b/parquet-avro/src/test/resources/all-minus-fixed.avsc
@@ -0,0 +1,65 @@
+{
+  "name" : "myrecord",
+  "namespace": "parquet.avro",
+  "type" : "record",
+  "fields" : [ {
+    "name" : "mynull",
+    "type" : "null"
+  }, {
+    "name" : "myboolean",
+    "type" : "boolean"
+  }, {
+    "name" : "myint",
+    "type" : "int"
+  }, {
+    "name" : "mylong",
+    "type" : "long"
+  }, {
+    "name" : "myfloat",
+    "type" : "float"
+  }, {
+    "name" : "mydouble",
+    "type" : "double"
+  }, {
+    "name" : "mybytes",
+    "type" : "bytes"
+  }, {
+    "name" : "mystring",
+    "type" : "string"
+  }, {
+    "name" : "mynestedrecord",
+    "type" : {
+      "type" : "record",
+      "name" : "ignored1",
+      "fields" : [ {
+        "name" : "mynestedint",
+        "type" : "int"
+      } ]
+    }
+  }, {
+    "name" : "myenum",
+    "type" : {
+      "type" : "enum",
+      "name" : "ignored2",
+      "symbols" : [ "a", "b" ]
+    }
+  }, {
+    "name" : "myarray",
+    "type" : {
+      "type" : "array",
+      "items" : "int"
+    }
+  }, {
+    "name" : "myoptionalarray",
+    "type" : [ "null", {
+       "type" : "array",
+       "items" : "int"
+    }]
+  }, {
+    "name" : "mymap",
+    "type" : {
+      "type" : "map",
+      "values" : "int"
+    }
+  } ]
+}
diff --git a/parquet-avro/src/test/resources/all.avsc b/parquet-avro/src/test/resources/all.avsc
new file mode 100644
index 0000000..1a09b01
--- /dev/null
+++ b/parquet-avro/src/test/resources/all.avsc
@@ -0,0 +1,66 @@
+{
+  "name" : "myrecord",
+  "namespace": "parquet.avro",
+  "type" : "record",
+  "fields" : [ {
+    "name" : "mynull",
+    "type" : "null"
+  }, {
+    "name" : "myboolean",
+    "type" : "boolean"
+  }, {
+    "name" : "myint",
+    "type" : "int"
+  }, {
+    "name" : "mylong",
+    "type" : "long"
+  }, {
+    "name" : "myfloat",
+    "type" : "float"
+  }, {
+    "name" : "mydouble",
+    "type" : "double"
+  }, {
+    "name" : "mybytes",
+    "type" : "bytes"
+  }, {
+    "name" : "mystring",
+    "type" : "string"
+  }, {
+    "name" : "mynestedrecord",
+    "type" : {
+      "type" : "record",
+      "name" : "ignored1",
+      "fields" : [ {
+        "name" : "mynestedint",
+        "type" : "int"
+      } ]
+    }
+  }, {
+    "name" : "myenum",
+    "type" : {
+      "type" : "enum",
+      "name" : "ignored2",
+      "symbols" : [ "a", "b" ]
+    }
+  }, {
+    "name" : "myarray",
+    "type" : {
+      "type" : "array",
+      "items" : "int"
+    }
+  }, {
+    "name" : "mymap",
+    "type" : {
+      "type" : "map",
+      "values" : "int"
+    }
+  }, {
+    "name" : "myfixed",
+    "type" : {
+      "type" : "fixed",
+      "name" : "ignored3",
+      "size" : 1
+    }
+  } ]
+}
diff --git a/parquet-avro/src/test/resources/car.avdl b/parquet-avro/src/test/resources/car.avdl
new file mode 100644
index 0000000..e330aae
--- /dev/null
+++ b/parquet-avro/src/test/resources/car.avdl
@@ -0,0 +1,38 @@
+@namespace("parquet.avro")
+protocol Cars {
+
+    record Service {
+        long date;
+        string mechanic;
+    }
+
+    record Stereo {
+        string make;
+        int speakers;
+    }
+
+    record LeatherTrim {
+        string colour;
+    }
+
+    enum EngineType {
+        DIESEL, PETROL
+    }
+
+    record Engine {
+        EngineType type;
+        float capacity;
+        boolean hasTurboCharger;
+    }
+
+    record Car {
+        long year;
+        string registration;
+        string make;
+        string model;
+        int doors;
+        Engine engine;
+        union { null, Stereo, LeatherTrim } optionalExtra = null;
+        array<Service> serviceHistory;
+    }
+}
\ No newline at end of file
diff --git a/parquet-cascading/.cache b/parquet-cascading/.cache
new file mode 100644
index 0000000..916a4dd
--- /dev/null
+++ b/parquet-cascading/.cache
Binary files differ
diff --git a/parquet-cascading/pom.xml b/parquet-cascading/pom.xml
new file mode 100644
index 0000000..81a3ead
--- /dev/null
+++ b/parquet-cascading/pom.xml
@@ -0,0 +1,88 @@
+<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.0.0-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-cascading</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Parquet Cascading</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <repositories>
+    <repository>
+     <id>conjars.org</id>
+      <url>http://conjars.org/repo</url>
+    </repository>
+  </repositories>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+        <groupId>com.twitter</groupId>
+        <artifactId>parquet-thrift</artifactId>
+        <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <version>1.2.17</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+       <groupId>cascading</groupId>
+       <artifactId>cascading-hadoop</artifactId>
+       <version>2.1.5</version>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java b/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java
new file mode 100644
index 0000000..0a79e77
--- /dev/null
+++ b/parquet-cascading/src/main/java/parquet/cascading/ParquetTBaseScheme.java
@@ -0,0 +1,66 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.cascading;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.thrift.TBase;
+
+import parquet.hadoop.ParquetInputFormat;
+import parquet.hadoop.mapred.DeprecatedParquetInputFormat;
+import parquet.hadoop.thrift.ThriftReadSupport;
+import parquet.thrift.ThriftRecordConverter;
+import cascading.flow.FlowProcess;
+import cascading.scheme.SinkCall;
+import cascading.tap.Tap;
+
+public class ParquetTBaseScheme<T extends TBase> extends ParquetValueScheme<T> {
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void sinkConfInit(FlowProcess<JobConf> arg0,
+      Tap<JobConf, RecordReader, OutputCollector> arg1, JobConf arg2) {
+    throw new UnsupportedOperationException("ParquetTBaseScheme does not support Sinks");
+
+  }
+
+  /**
+   * TODO: currently we cannot write Parquet files from TBase objects.
+   * All the underlying stuff exists, just need to link it.
+   */
+  @Override
+  public boolean isSink() { return false; }
+
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void sourceConfInit(FlowProcess<JobConf> fp,
+      Tap<JobConf, RecordReader, OutputCollector> tap, JobConf jobConf) {
+    jobConf.setInputFormat(DeprecatedParquetInputFormat.class);
+    ParquetInputFormat.setReadSupportClass(jobConf, ThriftReadSupport.class);
+    ThriftReadSupport.setRecordConverterClass(jobConf, ThriftRecordConverter.class);
+  }
+
+
+  @Override
+  public void sink(FlowProcess<JobConf> arg0, SinkCall<Object[], OutputCollector> arg1)
+      throws IOException {
+    throw new UnsupportedOperationException("ParquetTBaseScheme does not support Sinks");
+  }
+}
diff --git a/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java b/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
new file mode 100644
index 0000000..d75b6e7
--- /dev/null
+++ b/parquet-cascading/src/main/java/parquet/cascading/ParquetTupleScheme.java
@@ -0,0 +1,140 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.cascading;
+
+ import java.io.IOException;
+ import java.util.List;
+ import org.apache.hadoop.mapred.JobConf;
+ import org.apache.hadoop.mapred.OutputCollector;
+ import org.apache.hadoop.mapred.RecordReader;
+ import org.apache.hadoop.fs.Path;
+
+ import parquet.hadoop.ParquetInputFormat;
+ import parquet.hadoop.ParquetFileReader;
+ import parquet.hadoop.Footer;
+ import parquet.hadoop.metadata.ParquetMetadata;
+ import parquet.hadoop.mapred.Container;
+ import parquet.hadoop.mapred.DeprecatedParquetInputFormat;
+ import parquet.schema.MessageType;
+
+ import cascading.flow.FlowProcess;
+ import cascading.scheme.SinkCall;
+ import cascading.scheme.Scheme;
+ import cascading.scheme.SourceCall;
+ import cascading.tap.Tap;
+ import cascading.tap.TapException;
+ import cascading.tap.CompositeTap;
+ import cascading.tap.hadoop.Hfs;
+ import cascading.tuple.Tuple;
+ import cascading.tuple.Fields;
+
+ /**
+  * A Cascading Scheme that converts Parquet groups into Cascading tuples.
+  * If you provide it with sourceFields, it will selectively materialize only the columns for those fields.
+  * The names must match the names in the Parquet schema.
+  * If you do not provide sourceFields, or use Fields.ALL or Fields.UNKNOWN, it will create one from the
+  * Parquet schema.
+  * Currently, only primitive types are supported. TODO: allow nested fields in the Parquet schema to be
+  * flattened to a top-level field in the Cascading tuple.
+  *
+  * @author Avi Bryant
+  */
+
+public class ParquetTupleScheme extends Scheme<JobConf, RecordReader, OutputCollector, Object[], Object[]>{
+
+  private static final long serialVersionUID = 0L;
+
+  public ParquetTupleScheme() {
+    super();
+  }
+
+  public ParquetTupleScheme(Fields sourceFields) {
+    super(sourceFields);
+  }
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void sourceConfInit(FlowProcess<JobConf> fp,
+      Tap<JobConf, RecordReader, OutputCollector> tap, JobConf jobConf) {
+    jobConf.setInputFormat(DeprecatedParquetInputFormat.class);
+    ParquetInputFormat.setReadSupportClass(jobConf, TupleReadSupport.class);
+    TupleReadSupport.setRequestedFields(jobConf, getSourceFields());
+ }
+
+ @Override
+ public Fields retrieveSourceFields(FlowProcess<JobConf> flowProcess, Tap tap) {
+    MessageType schema = readSchema(flowProcess, tap);
+    SchemaIntersection intersection = new SchemaIntersection(schema, getSourceFields());
+
+    setSourceFields(intersection.getSourceFields());
+
+    return getSourceFields();
+  }
+
+  private MessageType readSchema(FlowProcess<JobConf> flowProcess, Tap tap) {
+    try {
+      Hfs hfs;
+
+      if( tap instanceof CompositeTap )
+        hfs = (Hfs) ( (CompositeTap) tap ).getChildTaps().next();
+      else
+        hfs = (Hfs) tap;
+
+      List<Footer> footers = ParquetFileReader.readFooters(flowProcess.getConfigCopy(), hfs.getPath());
+      if(footers.isEmpty()) {
+        throw new TapException("Could not read Parquet metadata at " + hfs.getPath());
+      } else {
+        return footers.get(0).getParquetMetadata().getFileMetaData().getSchema();
+      }
+    } catch (IOException e) {
+      throw new TapException(e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public boolean source(FlowProcess<JobConf> fp, SourceCall<Object[], RecordReader> sc)
+      throws IOException {
+    Container<Tuple> value = (Container<Tuple>) sc.getInput().createValue();
+    boolean hasNext = sc.getInput().next(null, value);
+    if (!hasNext) { return false; }
+
+    // Skip nulls
+    if (value == null) { return true; }
+
+    sc.getIncomingEntry().setTuple(value.get());
+    return true;
+  }
+
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void sinkConfInit(FlowProcess<JobConf> arg0,
+      Tap<JobConf, RecordReader, OutputCollector> arg1, JobConf arg2) {
+    throw new UnsupportedOperationException("ParquetTupleScheme does not support Sinks");
+
+  }
+
+  @Override
+  public boolean isSink() { return false; }
+
+
+  @Override
+  public void sink(FlowProcess<JobConf> arg0, SinkCall<Object[], OutputCollector> arg1)
+      throws IOException {
+    throw new UnsupportedOperationException("ParquetTupleScheme does not support Sinks");
+  }
+}
\ No newline at end of file
diff --git a/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java b/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java
new file mode 100644
index 0000000..1492766
--- /dev/null
+++ b/parquet-cascading/src/main/java/parquet/cascading/ParquetValueScheme.java
@@ -0,0 +1,57 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.cascading;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
+
+import parquet.hadoop.mapred.Container;
+
+import cascading.flow.FlowProcess;
+import cascading.scheme.Scheme;
+import cascading.scheme.SourceCall;
+import cascading.tuple.Tuple;
+
+/**
+ * A Cascading Scheme that returns a simple Tuple with a single value, the "value" object
+ * coming out of the underlying InputFormat.
+ *
+ * This is an abstract class; implementations are expected to set up their Input/Output Formats
+ * correctly in the respective Init methods.
+ */
+public abstract class ParquetValueScheme<T> extends Scheme<JobConf, RecordReader, OutputCollector, Object[], Object[]>{
+
+  private static final long serialVersionUID = 157560846420730043L;
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public boolean source(FlowProcess<JobConf> fp, SourceCall<Object[], RecordReader> sc)
+      throws IOException {
+    Container<T> value = (Container<T>) sc.getInput().createValue();
+    boolean hasNext = sc.getInput().next(null, value);
+    if (!hasNext) { return false; }
+
+    // Skip nulls
+    if (value == null) { return true; }
+
+    sc.getIncomingEntry().setTuple(new Tuple(value.get()));
+    return true;
+  }
+
+}
diff --git a/parquet-cascading/src/main/java/parquet/cascading/SchemaIntersection.java b/parquet-cascading/src/main/java/parquet/cascading/SchemaIntersection.java
new file mode 100644
index 0000000..10d647d
--- /dev/null
+++ b/parquet-cascading/src/main/java/parquet/cascading/SchemaIntersection.java
@@ -0,0 +1,60 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.cascading;
+
+import parquet.schema.MessageType;
+import parquet.schema.Type;
+
+import cascading.tuple.Fields;
+
+import java.util.List;
+import java.util.ArrayList;
+
+public class SchemaIntersection {
+
+  private final MessageType requestedSchema;
+  private final Fields sourceFields;
+
+  public SchemaIntersection(MessageType fileSchema, Fields requestedFields) {
+    if(requestedFields == Fields.UNKNOWN)
+      requestedFields = Fields.ALL;
+
+    Fields newFields = Fields.NONE;
+    List<Type> newSchemaFields = new ArrayList<Type>();
+    int schemaSize = fileSchema.getFieldCount();
+
+    for (int i = 0; i < schemaSize; i++) {
+      Type type = fileSchema.getType(i);
+      Fields name = new Fields(type.getName());
+
+      if(requestedFields.contains(name)) {
+        newFields = newFields.append(name);
+        newSchemaFields.add(type);
+      }
+    }
+
+    this.sourceFields = newFields;
+    this.requestedSchema = new MessageType(fileSchema.getName(), newSchemaFields);
+  }
+
+  public MessageType getRequestedSchema() {
+    return requestedSchema;
+  }
+
+  public Fields getSourceFields() {
+    return sourceFields;
+  }
+}
\ No newline at end of file
diff --git a/parquet-cascading/src/main/java/parquet/cascading/TupleReadSupport.java b/parquet-cascading/src/main/java/parquet/cascading/TupleReadSupport.java
new file mode 100644
index 0000000..647d233
--- /dev/null
+++ b/parquet-cascading/src/main/java/parquet/cascading/TupleReadSupport.java
@@ -0,0 +1,77 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.cascading;
+
+import java.util.Map;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.commons.lang.StringUtils;
+
+import cascading.tuple.Tuple;
+import cascading.tuple.Fields;
+import cascading.flow.hadoop.util.HadoopUtil;
+
+import parquet.hadoop.api.ReadSupport;
+import parquet.io.api.RecordMaterializer;
+import parquet.schema.MessageType;
+import parquet.cascading.convert.TupleRecordMaterializer;
+
+
+public class TupleReadSupport extends ReadSupport<Tuple> {
+  static final String PARQUET_CASCADING_REQUESTED_FIELDS = "parquet.cascading.requested.fields";
+
+  static protected Fields getRequestedFields(Configuration configuration) {
+    String fieldsString = configuration.get(PARQUET_CASCADING_REQUESTED_FIELDS);
+
+    if(fieldsString == null)
+      return Fields.ALL;
+
+    String[] parts = StringUtils.split(fieldsString, ":");
+    if(parts.length == 0)
+      return Fields.ALL;
+    else
+      return new Fields(parts);
+  }
+
+  static protected void setRequestedFields(JobConf configuration, Fields fields) {
+    String fieldsString = StringUtils.join(fields.iterator(), ":");
+    configuration.set(PARQUET_CASCADING_REQUESTED_FIELDS, fieldsString);
+  }
+
+  @Override
+  public ReadContext init(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema) {
+    Fields requestedFields = getRequestedFields(configuration);
+    if (requestedFields == null) {
+      return new ReadContext(fileSchema);
+    } else {
+      SchemaIntersection intersection = new SchemaIntersection(fileSchema, requestedFields);
+      return new ReadContext(intersection.getRequestedSchema());
+    }
+  }
+
+  @Override
+  public RecordMaterializer<Tuple> prepareForRead(
+      Configuration configuration,
+      Map<String, String> keyValueMetaData,
+      MessageType fileSchema,
+      ReadContext readContext) {
+    MessageType requestedSchema = readContext.getRequestedSchema();
+    return new TupleRecordMaterializer(requestedSchema);
+  }
+
+}
diff --git a/parquet-cascading/src/main/java/parquet/cascading/convert/TupleConverter.java b/parquet-cascading/src/main/java/parquet/cascading/convert/TupleConverter.java
new file mode 100644
index 0000000..4f74c8c
--- /dev/null
+++ b/parquet-cascading/src/main/java/parquet/cascading/convert/TupleConverter.java
@@ -0,0 +1,112 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.cascading.convert;
+
+import cascading.tuple.Tuple;
+
+import parquet.io.ParquetDecodingException;
+import parquet.io.api.Binary;
+import parquet.io.api.Converter;
+import parquet.io.api.GroupConverter;
+import parquet.io.api.PrimitiveConverter;
+import parquet.pig.TupleConversionException;
+import parquet.schema.GroupType;
+import parquet.schema.OriginalType;
+import parquet.schema.PrimitiveType;
+import parquet.schema.Type;
+import parquet.schema.Type.Repetition;
+
+public class TupleConverter extends GroupConverter {
+
+  protected Tuple currentTuple;
+  private final Converter[] converters;
+
+  public TupleConverter(GroupType parquetSchema) {
+    int schemaSize = parquetSchema.getFieldCount();
+
+    this.converters = new Converter[schemaSize];
+    for (int i = 0; i < schemaSize; i++) {
+      Type type = parquetSchema.getType(i);
+      converters[i] = newConverter(type, i);
+    }
+  }
+
+  private Converter newConverter(Type type, int i) {
+    if(!type.isPrimitive()) {
+      throw new IllegalArgumentException("cascading can only build tuples from primitive types");
+    } else {
+      return new TuplePrimitiveConverter(this, i);
+    }
+  }
+
+  @Override
+  public Converter getConverter(int fieldIndex) {
+    return converters[fieldIndex];
+  }
+
+  @Override
+  final public void start() {
+    currentTuple = Tuple.size(converters.length);
+  }
+
+  @Override
+  public void end() {
+  }
+
+  final public Tuple getCurrentTuple() {
+    return currentTuple;
+  }
+
+  static final class TuplePrimitiveConverter extends PrimitiveConverter {
+    private final TupleConverter parent;
+    private final int index;
+
+    public TuplePrimitiveConverter(TupleConverter parent, int index) {
+      this.parent = parent;
+      this.index = index;
+    }
+
+    @Override
+    public void addBinary(Binary value) {
+      parent.getCurrentTuple().setString(index, value.toStringUsingUTF8());
+    }
+
+    @Override
+    public void addBoolean(boolean value) {
+      parent.getCurrentTuple().setBoolean(index, value);
+    }
+
+    @Override
+    public void addDouble(double value) {
+      parent.getCurrentTuple().setDouble(index, value);
+    }
+
+    @Override
+    public void addFloat(float value) {
+      parent.getCurrentTuple().setFloat(index, value);
+    }
+
+    @Override
+    public void addInt(int value) {
+      parent.getCurrentTuple().setInteger(index, value);
+    }
+
+    @Override
+    public void addLong(long value) {
+      parent.getCurrentTuple().setLong(index, value);
+    }
+  }
+}
\ No newline at end of file
diff --git a/parquet-cascading/src/main/java/parquet/cascading/convert/TupleRecordMaterializer.java b/parquet-cascading/src/main/java/parquet/cascading/convert/TupleRecordMaterializer.java
new file mode 100644
index 0000000..faeefbc
--- /dev/null
+++ b/parquet-cascading/src/main/java/parquet/cascading/convert/TupleRecordMaterializer.java
@@ -0,0 +1,43 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.cascading.convert;
+
+import cascading.tuple.Tuple;
+import cascading.tuple.Fields;
+
+import parquet.io.api.GroupConverter;
+import parquet.io.api.RecordMaterializer;
+import parquet.schema.GroupType;
+
+public class TupleRecordMaterializer extends RecordMaterializer<Tuple> {
+
+  private TupleConverter root;
+
+  public TupleRecordMaterializer(GroupType parquetSchema) {
+    this.root = new TupleConverter(parquetSchema);
+  }
+
+  @Override
+  public Tuple getCurrentRecord() {
+    return root.getCurrentTuple();
+  }
+
+  @Override
+  public GroupConverter getRootConverter() {
+    return root;
+  }
+
+}
\ No newline at end of file
diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml
index d67599b..42aee96 100644
--- a/parquet-column/pom.xml
+++ b/parquet-column/pom.xml
@@ -20,6 +20,16 @@
 
   <dependencies>
     <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-encoding</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
       <version>1.7</version>
@@ -37,6 +47,13 @@
       <version>0.1.0</version>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-encoding</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
@@ -44,6 +61,14 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
         <executions>
           <execution>
             <goals>
diff --git a/parquet-column/src/main/java/parquet/bytes/BytesUtils.java b/parquet-column/src/main/java/parquet/bytes/BytesUtils.java
deleted file mode 100644
index 91f06b3..0000000
--- a/parquet-column/src/main/java/parquet/bytes/BytesUtils.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Copyright 2012 Twitter, Inc.
- *
- * Licensed 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.bytes;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import parquet.Log;
-
-/**
- * utility methods to deal with bytes
- *
- * @author Julien Le Dem
- *
- */
-public class BytesUtils {
-  private static final Log LOG = Log.getLog(BytesUtils.class);
-
-  public static final Charset UTF8 = Charset.forName("UTF-8");
-
-  /**
-   * give the number of bits needed to encode an int given the max value
-   * @param bound max int that we want to encode
-   * @return the number of bits required
-   */
-  public static int getWidthFromMaxInt(int bound) {
-    return 32 - Integer.numberOfLeadingZeros(bound);
-  }
-
-  /**
-   * reads an int in little endian at the given position
-   * @param in
-   * @param offset
-   * @return
-   * @throws IOException
-   */
-  public static int readIntLittleEndian(byte[] in, int offset) throws IOException {
-    int ch4 = in[offset] & 0xff;
-    int ch3 = in[offset + 1] & 0xff;
-    int ch2 = in[offset + 2] & 0xff;
-    int ch1 = in[offset + 3] & 0xff;
-    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
-  }
-
-  public static int readIntLittleEndian(InputStream in) throws IOException {
-    // TODO: this is duplicated code in LittleEndianDataInputStream
-    int ch1 = in.read();
-    int ch2 = in.read();
-    int ch3 = in.read();
-    int ch4 = in.read();
-    if ((ch1 | ch2 | ch3 | ch4) < 0)
-        throw new EOFException();
-    if (Log.DEBUG) LOG.debug("read le int: " + ch1 + " " + ch2 + " " + ch3 + " " + ch4 + " => " + ((ch4 << 24) + (ch3 << 16) + (ch2 << 8) + (ch1 << 0)));
-    return ((ch4 << 24) + (ch3 << 16) + (ch2 << 8) + (ch1 << 0));
-  }
-
-  public static void writeIntLittleEndian(OutputStream out, int v) throws IOException {
-    // TODO: this is duplicated code in LittleEndianDataOutputStream
-    out.write((v >>>  0) & 0xFF);
-    out.write((v >>>  8) & 0xFF);
-    out.write((v >>> 16) & 0xFF);
-    out.write((v >>> 24) & 0xFF);
-    if (Log.DEBUG) LOG.debug("write le int: " + v + " => "+ ((v >>>  0) & 0xFF) + " " + ((v >>>  8) & 0xFF) + " " + ((v >>> 16) & 0xFF) + " " + ((v >>> 24) & 0xFF));
-  }
-
-  public static int readUnsignedVarInt(InputStream in) throws IOException {
-    int value = 0;
-    int i = 0;
-    int b;
-    while (((b = in.read()) & 0x80) != 0) {
-      value |= (b & 0x7F) << i;
-      i += 7;
-    }
-    return value | (b << i);
-  }
-
-  public static void writeUnsignedVarInt(int value, OutputStream out) throws IOException {
-    while ((value & 0xFFFFFF80) != 0L) {
-      out.write((value & 0x7F) | 0x80);
-      value >>>= 7;
-    }
-    out.write(value & 0x7F);
-  }
-}
diff --git a/parquet-column/src/main/java/parquet/column/ColumnReader.java b/parquet-column/src/main/java/parquet/column/ColumnReader.java
index a26127e..149250f 100644
--- a/parquet-column/src/main/java/parquet/column/ColumnReader.java
+++ b/parquet-column/src/main/java/parquet/column/ColumnReader.java
@@ -21,10 +21,18 @@
  * Reader for (repetition level, definition level, values) triplets.
  * At any given point in time, a ColumnReader points to a single (r, d, v) triplet.
  * In order to move to the next triplet, call {@link #consume()}.
- * 
- * Each iteration looks at the current definition level and value as well as the next
- * repetition level. TODO(julien): what does this mean? It's not clear. -todd
  *
+ * Depending on the type and the encoding of the column only a subset of the get* methods are implemented.
+ * Dictionary specific methods enable the upper layers to read the dictionary IDs without decoding the data.
+ * In particular the Converter will decode the strings in the dictionary only once and iterate on the
+ * dictionary IDs instead of the values.
+ *
+ * <ul>Each iteration looks at the current definition level and value as well as the next
+ * repetition level:
+ *  <li> The current definition level defines if the value is null.</li>
+ *  <li> If the value is defined we can read it with the correct get*() method.</li>
+ *  <li> Looking ahead to the next repetition determines what is the next column to read for in the FSA.</li>
+ * </ul>
  * @author Julien Le Dem
   */
 public interface ColumnReader {
@@ -51,9 +59,15 @@
   int getCurrentDefinitionLevel();
 
   /**
-   * @return the current value
+   * writes the current value to the converter
    */
-  String getString();
+  void writeCurrentValueToConverter();
+
+  /**
+   * available when the underlying encoding is dictionary based
+   * @return the dictionary id for the current value
+   */
+  int getCurrentValueDictionaryID();
 
   /**
    * @return the current value
@@ -84,4 +98,14 @@
    * @return the current value
    */
   double getDouble();
+
+  /**
+   * @return Descriptor of the column.
+   */
+  ColumnDescriptor getDescriptor();
+
+  /**
+   * Skip the current value
+   */
+  void skip();
 }
diff --git a/parquet-column/src/main/java/parquet/column/Dictionary.java b/parquet-column/src/main/java/parquet/column/Dictionary.java
new file mode 100644
index 0000000..ab709d9
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/Dictionary.java
@@ -0,0 +1,63 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column;
+
+import parquet.io.api.Binary;
+
+/**
+ * a dictionary to decode dictionary based encodings
+ *
+ * @author Julien Le Dem
+ *
+ */
+public abstract class Dictionary {
+
+  private final Encoding encoding;
+
+  public Dictionary(Encoding encoding) {
+    this.encoding = encoding;
+  }
+
+  public Encoding getEncoding() {
+    return encoding;
+  }
+
+  public abstract int getMaxId();
+
+  public Binary decodeToBinary(int id) {
+    throw new UnsupportedOperationException(this.getClass().getName());
+  }
+
+  public int decodeToInt(int id) {
+    throw new UnsupportedOperationException(this.getClass().getName());
+  }
+
+  public long decodeToLong(int id) {
+    throw new UnsupportedOperationException(this.getClass().getName());
+  }
+
+  public float decodeToFloat(int id) {
+    throw new UnsupportedOperationException(this.getClass().getName());
+  }
+
+  public double decodeToDouble(int id) {
+    throw new UnsupportedOperationException(this.getClass().getName());
+  }
+
+  public boolean decodeToBoolean(int id) {
+    throw new UnsupportedOperationException(this.getClass().getName());
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/Encoding.java b/parquet-column/src/main/java/parquet/column/Encoding.java
index 1758a3f..9cb8031 100644
--- a/parquet-column/src/main/java/parquet/column/Encoding.java
+++ b/parquet-column/src/main/java/parquet/column/Encoding.java
@@ -15,16 +15,29 @@
  */
 package parquet.column;
 
+import java.io.IOException;
+
+import parquet.bytes.BytesUtils;
+import parquet.column.page.DictionaryPage;
 import parquet.column.values.ValuesReader;
-import parquet.column.values.ValuesType;
-import parquet.column.values.bitpacking.BitPackingValuesReader;
-import parquet.column.values.boundedint.BoundedIntValuesFactory;
+import parquet.column.values.bitpacking.ByteBitPackingValuesReader;
+import parquet.column.values.boundedint.ZeroIntegerValuesReader;
+import parquet.column.values.dictionary.DictionaryValuesReader;
+import parquet.column.values.dictionary.PlainBinaryDictionary;
+import parquet.column.values.plain.BinaryPlainValuesReader;
 import parquet.column.values.plain.BooleanPlainValuesReader;
-import parquet.column.values.plain.PlainValuesReader;
+import parquet.column.values.plain.PlainValuesReader.DoublePlainValuesReader;
+import parquet.column.values.plain.PlainValuesReader.FloatPlainValuesReader;
+import parquet.column.values.plain.PlainValuesReader.IntegerPlainValuesReader;
+import parquet.column.values.plain.PlainValuesReader.LongPlainValuesReader;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesReader;
 import parquet.io.ParquetDecodingException;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+
+import static parquet.column.values.bitpacking.Packer.BIG_ENDIAN;
 
 /**
- * endoding of the data
+ * encoding of the data
  *
  * @author Julien Le Dem
  *
@@ -37,42 +50,78 @@
       switch (descriptor.getType()) {
       case BOOLEAN:
         return new BooleanPlainValuesReader();
+      case BINARY:
+        return new BinaryPlainValuesReader();
+      case FLOAT:
+        return new FloatPlainValuesReader();
+      case DOUBLE:
+        return new DoublePlainValuesReader();
+      case INT32:
+        return new IntegerPlainValuesReader();
+      case INT64:
+        return new LongPlainValuesReader();
       default:
-        return new PlainValuesReader();
+        throw new ParquetDecodingException("no plain reader for type " + descriptor.getType());
       }
     }
   },
 
+  /**
+   * Actually a combination of bit packing and run length encoding.
+   * TODO: Should we rename this to be more clear?
+   */
   RLE {
     @Override
     public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType) {
-      return BoundedIntValuesFactory.getBoundedReader(getMaxLevel(descriptor, valuesType));
+      int bitWidth = BytesUtils.getWidthFromMaxInt(getMaxLevel(descriptor, valuesType));
+      if(bitWidth == 0) {
+        return new ZeroIntegerValuesReader();
+      }
+      return new RunLengthBitPackingHybridValuesReader(bitWidth);
     }
   },
 
+  /**
+   * This is no longer used, and has been replaced by {@link #RLE}
+   * which is combination of bit packing and rle
+   */
+  @Deprecated
   BIT_PACKED {
     @Override
     public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType) {
-      return new BitPackingValuesReader(getMaxLevel(descriptor, valuesType));
+      return new ByteBitPackingValuesReader(getMaxLevel(descriptor, valuesType), BIG_ENDIAN);
     }
   },
 
   GROUP_VAR_INT {
-
     @Override // TODO: GROUP VAR INT encoding
-    public ValuesReader getValuesReader(ColumnDescriptor descriptor,
-        ValuesType valuesType) {
+    public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType) {
       throw new UnsupportedOperationException("NYI");
     }
-
   },
 
   PLAIN_DICTIONARY {
+    @Override
+    public ValuesReader getDictionaryBasedValuesReader(ColumnDescriptor descriptor, ValuesType valuesType, Dictionary dictionary) {
+      switch (descriptor.getType()) {
+      case BINARY:
+        return new DictionaryValuesReader(dictionary);
+      default:
+        throw new ParquetDecodingException("Dictionary encoding not supported for type: " + descriptor.getType());
+      }
+    }
 
-    @Override // TODO: dictionary encoding
-    public ValuesReader getValuesReader(ColumnDescriptor descriptor,
-        ValuesType valuesType) {
-      throw new UnsupportedOperationException("NYI");
+    @Override
+    public Dictionary initDictionary(ColumnDescriptor descriptor, DictionaryPage dictionaryPage) throws IOException {
+      if (descriptor.getType() != PrimitiveTypeName.BINARY) {
+        throw new UnsupportedOperationException("only Binary dictionaries are supported for now");
+      }
+      return new PlainBinaryDictionary(dictionaryPage);
+    }
+
+    @Override
+    public boolean usesDictionary() {
+      return true;
     }
 
   };
@@ -93,9 +142,44 @@
   }
 
   /**
+   * @return whether this encoding requires a dictionary
+   */
+  public boolean usesDictionary() {
+    return false;
+  }
+
+  /**
+   * initializes a dictionary from a page
+   * @param dictionaryPage
+   * @return the corresponding dictionary
+   */
+  public Dictionary initDictionary(ColumnDescriptor descriptor, DictionaryPage dictionaryPage) throws IOException {
+    throw new UnsupportedOperationException(this.name() + " does not support dictionary");
+  }
+
+  /**
+   * To read decoded values that don't require a dictionary
+   *
    * @param descriptor the column to read
    * @param valuesType the type of values
    * @return the proper values reader for the given column
+   * @throw {@link UnsupportedOperationException} if the encoding is dictionary based
    */
-  abstract public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType);
+  public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType) {
+    throw new UnsupportedOperationException("Error decoding " + descriptor + ". " + this.name() + " is dictionary based");
+  }
+
+  /**
+   * To read decoded values that require a dictionary
+   *
+   * @param descriptor the column to read
+   * @param valuesType the type of values
+   * @param dictionary the dictionary
+   * @return the proper values reader for the given column
+   * @throw {@link UnsupportedOperationException} if the encoding is not dictionary based
+   */
+  public ValuesReader getDictionaryBasedValuesReader(ColumnDescriptor descriptor, ValuesType valuesType, Dictionary dictionary) {
+    throw new UnsupportedOperationException(this.name() + " is not dictionary based");
+  }
+
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesType.java b/parquet-column/src/main/java/parquet/column/ValuesType.java
similarity index 95%
rename from parquet-column/src/main/java/parquet/column/values/ValuesType.java
rename to parquet-column/src/main/java/parquet/column/ValuesType.java
index 0566303..dadcd9c 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesType.java
+++ b/parquet-column/src/main/java/parquet/column/ValuesType.java
@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.values;
+package parquet.column;
 
 /**
  * The different type of values we can store in columns
diff --git a/parquet-column/src/main/java/parquet/column/impl/ColumnReadStoreImpl.java b/parquet-column/src/main/java/parquet/column/impl/ColumnReadStoreImpl.java
index c606a7e..abf3c1b 100644
--- a/parquet-column/src/main/java/parquet/column/impl/ColumnReadStoreImpl.java
+++ b/parquet-column/src/main/java/parquet/column/impl/ColumnReadStoreImpl.java
@@ -15,195 +15,65 @@
  */
 package parquet.column.impl;
 
-import java.io.IOException;
-
 import parquet.column.ColumnDescriptor;
 import parquet.column.ColumnReadStore;
 import parquet.column.ColumnReader;
 import parquet.column.page.PageReadStore;
 import parquet.column.page.PageReader;
-import parquet.io.ParquetDecodingException;
-import parquet.io.api.Binary;
+import parquet.io.api.Converter;
+import parquet.io.api.GroupConverter;
+import parquet.io.api.PrimitiveConverter;
+import parquet.schema.GroupType;
+import parquet.schema.MessageType;
+import parquet.schema.Type;
 
-
+/**
+ * Implementation of the ColumnReadStore
+ *
+ * Initializes individual columns based on schema and converter
+ *
+ * @author Julien Le Dem
+ *
+ */
 public class ColumnReadStoreImpl implements ColumnReadStore {
 
   private final PageReadStore pageReadStore;
+  private final GroupConverter recordConverter;
+  private final MessageType schema;
 
-  public ColumnReadStoreImpl(PageReadStore pageReadStore) {
+  /**
+   * @param pageReadStore uderlying page storage
+   * @param recordConverter the user provided converter to materialize records
+   * @param schema the schema we are reading
+   */
+  public ColumnReadStoreImpl(PageReadStore pageReadStore, GroupConverter recordConverter, MessageType schema) {
     super();
     this.pageReadStore = pageReadStore;
+    this.recordConverter = recordConverter;
+    this.schema = schema;
   }
 
+  @Override
   public ColumnReader getColumnReader(ColumnDescriptor path) {
     return newMemColumnReader(path, pageReadStore.getPageReader(path));
   }
 
   private ColumnReaderImpl newMemColumnReader(ColumnDescriptor path, PageReader pageReader) {
-    switch (path.getType()) {
-    case INT32:
-      return new INT32MemColumnReader(path, pageReader);
-    case INT64:
-      return new INT64MemColumnReader(path, pageReader);
-    case BOOLEAN:
-      return new BOOLEANMemColumnReader(path, pageReader);
-    case BINARY:
-      return new BINARYMemColumnReader(path, pageReader);
-    case FLOAT:
-      return new FLOATMemColumnReader(path, pageReader);
-    case DOUBLE:
-      return new DOUBLEMemColumnReader(path, pageReader);
-    }
-    throw new ParquetDecodingException("type "+path.getType()+" not supported");
+    PrimitiveConverter converter = getPrimitiveConverter(path);
+    return new ColumnReaderImpl(path, pageReader, converter);
   }
 
-  private static final class INT32MemColumnReader extends ColumnReaderImpl {
-    private int currentInt;
-
-    public INT32MemColumnReader(ColumnDescriptor path, PageReader pageReader) {
-      super(path, pageReader);
+  private PrimitiveConverter getPrimitiveConverter(ColumnDescriptor path) {
+    Type currentType = schema;
+    Converter currentConverter = recordConverter;
+    for (String fieldName : path.getPath()) {
+      final GroupType groupType = currentType.asGroupType();
+      int fieldIndex = groupType.getFieldIndex(fieldName);
+      currentType = groupType.getType(fieldName);
+      currentConverter = currentConverter.asGroupConverter().getConverter(fieldIndex);
     }
-
-    @Override
-    public int getInteger() {
-      checkValueRead();
-      return currentInt;
-    }
-
-    @Override
-    protected void readCurrentValue() {
-        currentInt = dataColumn.readInteger();
-    }
-
-    @Override
-    public String getCurrentValueToString() throws IOException {
-      checkRead();
-      return String.valueOf(currentInt);
-    }
+    PrimitiveConverter converter = currentConverter.asPrimitiveConverter();
+    return converter;
   }
 
-  private static final class INT64MemColumnReader extends ColumnReaderImpl {
-    private long currentLong;
-
-    public INT64MemColumnReader(ColumnDescriptor path, PageReader pageReader) {
-      super(path, pageReader);
-    }
-
-    @Override
-    public long getLong() {
-      checkValueRead();
-      return currentLong;
-    }
-
-    @Override
-    protected void readCurrentValue() {
-      currentLong = dataColumn.readLong();
-    }
-
-    @Override
-    public String getCurrentValueToString() throws IOException {
-      checkRead();
-      return String.valueOf(currentLong);
-    }
-  }
-
-  private static final class BINARYMemColumnReader extends ColumnReaderImpl {
-    private Binary current;
-
-    public BINARYMemColumnReader(ColumnDescriptor path, PageReader pageReader) {
-      super(path, pageReader);
-    }
-
-    @Override
-    public Binary getBinary() {
-      checkValueRead();
-      return current;
-    }
-
-    @Override
-    protected void readCurrentValue() {
-      current = dataColumn.readBytes();
-    }
-
-    @Override
-    public String getCurrentValueToString() throws IOException {
-      checkRead();
-      return String.valueOf(current);
-    }
-  }
-
-  private static final class BOOLEANMemColumnReader extends ColumnReaderImpl {
-    private boolean current;
-
-    public BOOLEANMemColumnReader(ColumnDescriptor path, PageReader pageReader) {
-      super(path, pageReader);
-    }
-
-    @Override
-    public boolean getBoolean() {
-      checkValueRead();
-      return current;
-    }
-
-    @Override
-    protected void readCurrentValue() {
-      current = dataColumn.readBoolean();
-    }
-
-    @Override
-    public String getCurrentValueToString() throws IOException {
-      checkRead();
-      return String.valueOf(current);
-    }
-  }
-
-  private static final class FLOATMemColumnReader extends ColumnReaderImpl {
-    private float current;
-
-    public FLOATMemColumnReader(ColumnDescriptor path, PageReader pageReader) {
-      super(path, pageReader);
-    }
-
-    @Override
-    public float getFloat() {
-      checkValueRead();
-      return current;
-    }
-
-    @Override
-    protected void readCurrentValue() {
-      current = dataColumn.readFloat();
-    }
-
-    @Override
-    public String getCurrentValueToString() throws IOException {
-      checkRead();
-      return String.valueOf(current);
-    }
-  }
-
-  private static final class DOUBLEMemColumnReader extends ColumnReaderImpl {
-    private double current;
-
-    public DOUBLEMemColumnReader(ColumnDescriptor path, PageReader pageReader) {
-      super(path, pageReader);
-    }
-
-    @Override
-    public double getDouble() {
-      checkValueRead();
-      return current;
-    }
-
-    @Override
-    protected void readCurrentValue() {
-      current = dataColumn.readDouble();
-    }
-
-    @Override
-    public String getCurrentValueToString() throws IOException {
-      checkRead();
-      return String.valueOf(current);
-    }
-  }
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/impl/ColumnReaderImpl.java b/parquet-column/src/main/java/parquet/column/impl/ColumnReaderImpl.java
index e45b5de..3ece00b 100644
--- a/parquet-column/src/main/java/parquet/column/impl/ColumnReaderImpl.java
+++ b/parquet-column/src/main/java/parquet/column/impl/ColumnReaderImpl.java
@@ -15,19 +15,26 @@
  */
 package parquet.column.impl;
 
+import static java.lang.String.format;
 import static parquet.Log.DEBUG;
+import static parquet.Preconditions.checkNotNull;
 
 import java.io.IOException;
 
 import parquet.Log;
 import parquet.column.ColumnDescriptor;
 import parquet.column.ColumnReader;
+import parquet.column.Dictionary;
+import parquet.column.ValuesType;
+import parquet.column.page.DictionaryPage;
 import parquet.column.page.Page;
 import parquet.column.page.PageReader;
 import parquet.column.values.ValuesReader;
-import parquet.column.values.ValuesType;
 import parquet.io.ParquetDecodingException;
 import parquet.io.api.Binary;
+import parquet.io.api.PrimitiveConverter;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+import parquet.schema.PrimitiveType.PrimitiveTypeNameConverter;
 
 /**
  * ColumnReader implementation
@@ -35,12 +42,40 @@
  * @author Julien Le Dem
  *
  */
-abstract class ColumnReaderImpl implements ColumnReader {
+class ColumnReaderImpl implements ColumnReader {
   private static final Log LOG = Log.getLog(ColumnReaderImpl.class);
 
+  private static abstract class Binding {
+    abstract void read();
+    abstract void skip();
+    abstract void writeValue();
+    public int getDictionaryId() {
+      throw new UnsupportedOperationException();
+    }
+    public int getInteger() {
+      throw new UnsupportedOperationException();
+    }
+    public boolean getBoolean() {
+      throw new UnsupportedOperationException();
+    }
+    public long getLong() {
+      throw new UnsupportedOperationException();
+    }
+    public Binary getBinary() {
+      throw new UnsupportedOperationException();
+    }
+    public float getFloat() {
+      throw new UnsupportedOperationException();
+    }
+    public double getDouble() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
   private final ColumnDescriptor path;
   private final long totalValueCount;
   private final PageReader pageReader;
+  private final Dictionary dictionary;
 
   private ValuesReader repetitionLevelColumn;
   private ValuesReader definitionLevelColumn;
@@ -48,6 +83,7 @@
 
   private int repetitionLevel;
   private int definitionLevel;
+  private int dictionaryId;
   private boolean valueRead = false;
   private boolean consumed = true;
 
@@ -55,20 +91,199 @@
   private int readValuesInPage;
   private long pageValueCount;
 
+  private final PrimitiveConverter converter;
+  private Binding binding;
+
+  private void bindToDictionary(final Dictionary dictionary) {
+    binding =
+        new Binding() {
+          void read() {
+            dictionaryId = dataColumn.readValueDictionaryId();
+          }
+          public void skip() {
+            dataColumn.skip();
+          }
+          public int getDictionaryId() {
+            return dictionaryId;
+          }
+          void writeValue() {
+            converter.addValueFromDictionary(dictionaryId);
+          }
+          public int getInteger() {
+            return dictionary.decodeToInt(dictionaryId);
+          }
+          public boolean getBoolean() {
+            return dictionary.decodeToBoolean(dictionaryId);
+          }
+          public long getLong() {
+            return dictionary.decodeToLong(dictionaryId);
+          }
+          public Binary getBinary() {
+            return dictionary.decodeToBinary(dictionaryId);
+          }
+          public float getFloat() {
+            return dictionary.decodeToFloat(dictionaryId);
+          }
+          public double getDouble() {
+            return dictionary.decodeToDouble(dictionaryId);
+          }
+        };
+  }
+
+  private void bind(PrimitiveTypeName type) {
+    binding = type.convert(new PrimitiveTypeNameConverter<Binding>() {
+      @Override
+      public Binding convertFLOAT(PrimitiveTypeName primitiveTypeName) {
+        return new Binding() {
+          float current;
+          void read() {
+            current = dataColumn.readFloat();
+          }
+          public void skip() {
+            current = 0;
+            dataColumn.skip();
+          }
+          public float getFloat() {
+            return current;
+          }
+          void writeValue() {
+            converter.addFloat(current);
+          }
+        };
+      }
+      @Override
+      public Binding convertDOUBLE(PrimitiveTypeName primitiveTypeName) {
+        return new Binding() {
+          double current;
+          void read() {
+            current = dataColumn.readDouble();
+          }
+          public void skip() {
+            current = 0;
+            dataColumn.skip();
+          }
+          public double getDouble() {
+            return current;
+          }
+          void writeValue() {
+            converter.addDouble(current);
+          }
+        };
+      }
+      @Override
+      public Binding convertINT32(PrimitiveTypeName primitiveTypeName) {
+        return new Binding() {
+          int current;
+          void read() {
+            current = dataColumn.readInteger();
+          }
+          public void skip() {
+            current = 0;
+            dataColumn.skip();
+          }
+          @Override
+          public int getInteger() {
+            return current;
+          }
+          void writeValue() {
+            converter.addInt(current);
+          }
+        };
+      }
+      @Override
+      public Binding convertINT64(PrimitiveTypeName primitiveTypeName) {
+        return new Binding() {
+          long current;
+          void read() {
+            current = dataColumn.readLong();
+          }
+          public void skip() {
+            current = 0;
+            dataColumn.skip();
+          }
+          @Override
+          public long getLong() {
+            return current;
+          }
+          void writeValue() {
+            converter.addLong(current);
+          }
+        };
+      }
+      @Override
+      public Binding convertINT96(PrimitiveTypeName primitiveTypeName) {
+        throw new UnsupportedOperationException("INT96 NYI");
+      }
+      @Override
+      public Binding convertFIXED_LEN_BYTE_ARRAY(
+          PrimitiveTypeName primitiveTypeName) {
+        throw new UnsupportedOperationException("FIXED_LEN_BYTE_ARRAY NYI");
+      }
+      @Override
+      public Binding convertBOOLEAN(PrimitiveTypeName primitiveTypeName) {
+        return new Binding() {
+          boolean current;
+          void read() {
+            current = dataColumn.readBoolean();
+          }
+          public void skip() {
+            current = false;
+            dataColumn.skip();
+          }
+          @Override
+          public boolean getBoolean() {
+            return current;
+          }
+          void writeValue() {
+            converter.addBoolean(current);
+          }
+        };
+      }
+      @Override
+      public Binding convertBINARY(PrimitiveTypeName primitiveTypeName) {
+        return new Binding() {
+          Binary current;
+          void read() {
+            current = dataColumn.readBytes();
+          }
+          public void skip() {
+            current = null;
+            dataColumn.skip();
+          }
+          @Override
+          public Binary getBinary() {
+            return current;
+          }
+          void writeValue() {
+            converter.addBinary(current);
+          }
+        };
+      }
+    });
+  }
+
   /**
-   *
+   * creates a reader for triplets
    * @param path the descriptor for the corresponding column
    * @param pageReader the underlying store to read from
    */
-  public ColumnReaderImpl(ColumnDescriptor path, PageReader pageReader) {
-    if (path == null) {
-      throw new NullPointerException("path");
+  public ColumnReaderImpl(ColumnDescriptor path, PageReader pageReader, PrimitiveConverter converter) {
+    this.path = checkNotNull(path, "path");
+    this.pageReader = checkNotNull(pageReader, "pageReader");
+    this.converter = checkNotNull(converter, "converter");
+    DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
+    if (dictionaryPage != null) {
+      try {
+        this.dictionary = dictionaryPage.getEncoding().initDictionary(path, dictionaryPage);
+        if (converter.hasDictionarySupport()) {
+          converter.setDictionary(dictionary);
+        }
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not decode the dictionary for " + path, e);
+      }
+    } else {
+      this.dictionary = null;
     }
-    if (pageReader == null) {
-      throw new NullPointerException("pageReader");
-    }
-    this.path = path;
-    this.pageReader = pageReader;
     this.totalValueCount = pageReader.getTotalValueCount();
     if (totalValueCount == 0) {
       throw new ParquetDecodingException("totalValueCount == 0");
@@ -76,86 +291,91 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#isFullyConsumed()
    */
   @Override
   public boolean isFullyConsumed() {
-    return readValues >= totalValueCount;
+    return ((readValues == totalValueCount) && consumed) || (readValues > totalValueCount);
   }
 
   /**
-   *
    * {@inheritDoc}
-   * @see parquet.column.ColumnReader#getString()
+   * @see parquet.column.ColumnReader#writeCurrentValueToConverter()
    */
   @Override
-  public String getString() {
-    throw new UnsupportedOperationException();
+  public void writeCurrentValueToConverter() {
+    readIfPossible(false);
+    this.binding.writeValue();
+  }
+
+  @Override
+  public int getCurrentValueDictionaryID() {
+    readIfPossible(false);
+    return binding.getDictionaryId();
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getInteger()
    */
   @Override
   public int getInteger() {
-    throw new UnsupportedOperationException();
+    readIfPossible(false);
+    return this.binding.getInteger();
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getBoolean()
    */
   @Override
   public boolean getBoolean() {
-    throw new UnsupportedOperationException();
+    readIfPossible(false);
+    return this.binding.getBoolean();
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getLong()
    */
+  @Override
   public long getLong() {
-    throw new UnsupportedOperationException();
+    readIfPossible(false);
+    return this.binding.getLong();
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getBinary()
    */
   @Override
   public Binary getBinary() {
-    throw new UnsupportedOperationException();
+    readIfPossible(false);
+    return this.binding.getBinary();
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getFloat()
    */
   @Override
   public float getFloat() {
-    throw new UnsupportedOperationException();
+    readIfPossible(false);
+    return this.binding.getFloat();
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getDouble()
    */
   @Override
   public double getDouble() {
-    throw new UnsupportedOperationException();
+    readIfPossible(false);
+    return this.binding.getDouble();
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getCurrentRepetitionLevel()
    */
@@ -166,27 +386,55 @@
   }
 
   /**
-   * used for debugging
-   * @return
-   * @throws IOException
+   * {@inheritDoc}
+   * @see parquet.column.ColumnReader#getDescriptor()
    */
-  abstract public String getCurrentValueToString() throws IOException;
+  @Override
+  public ColumnDescriptor getDescriptor() {
+    return path;
+  }
 
   /**
    * reads the current value
    */
-  protected abstract void readCurrentValue();
+  public void readCurrentValue() {
+    binding.read();
+  }
 
-  protected void checkValueRead() {
-    checkRead();
-    if (!consumed && !valueRead) {
-      readCurrentValue();
-      valueRead = true;
+  /**
+   * Reads the value into the binding or skips forwards.
+   * @param skip If true do not deserialize just skip forwards
+   */
+  protected void readIfPossible(boolean skip) {
+    try {
+      checkRead();
+      if (!consumed && !valueRead) {
+        if ( skip ) {
+          binding.skip();
+        } else {
+          readCurrentValue();
+        }
+        valueRead = true;
+      }
+    } catch (RuntimeException e) {
+      throw new ParquetDecodingException(
+          format(
+              "Can't read value in column %s at value %d out of %d, %d out of %d in currentPage. repetition level: %d, definition level: %d",
+              path, readValues, totalValueCount, readValuesInPage, pageValueCount, repetitionLevel, definitionLevel),
+          e);
     }
   }
 
   /**
-   *
+   * {@inheritDoc}
+   * @see parquet.column.ColumnReader#skip()
+   */
+  @Override
+  public void skip() {
+    readIfPossible(true);
+  }
+
+  /**
    * {@inheritDoc}
    * @see parquet.column.ColumnReader#getCurrentDefinitionLevel()
    */
@@ -220,8 +468,20 @@
 
       this.repetitionLevelColumn = page.getRlEncoding().getValuesReader(path, ValuesType.REPETITION_LEVEL);
       this.definitionLevelColumn = page.getDlEncoding().getValuesReader(path, ValuesType.DEFINITION_LEVEL);
-      this.dataColumn = page.getValueEncoding().getValuesReader(path, ValuesType.VALUES);
-
+      if (page.getValueEncoding().usesDictionary()) {
+        if (dictionary == null) {
+          throw new ParquetDecodingException(
+              "could not read page " + page + " in col " + path + " as the dictionary was missing for encoding " + page.getValueEncoding());
+        }
+        this.dataColumn = page.getValueEncoding().getDictionaryBasedValuesReader(path, ValuesType.VALUES, dictionary);
+      } else {
+        this.dataColumn = page.getValueEncoding().getValuesReader(path, ValuesType.VALUES);
+      }
+      if (page.getValueEncoding().usesDictionary() && converter.hasDictionarySupport()) {
+        bindToDictionary(dictionary);
+      } else {
+        bind(path.getType());
+      }
       this.pageValueCount = page.getValueCount();
       this.readValuesInPage = 0;
       try {
diff --git a/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java b/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java
index ffcee8a..abd9373 100644
--- a/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java
+++ b/parquet-column/src/main/java/parquet/column/impl/ColumnWriteStoreImpl.java
@@ -33,11 +33,15 @@
   private final Map<ColumnDescriptor, ColumnWriterImpl> columns = new TreeMap<ColumnDescriptor, ColumnWriterImpl>();
   private final PageWriteStore pageWriteStore;
   private final int pageSizeThreshold;
+  private final boolean enableDictionary;
+  private final int initialSizePerCol;
 
-  public ColumnWriteStoreImpl(PageWriteStore pageWriteStore, int pageSizeThreshold) {
+  public ColumnWriteStoreImpl(PageWriteStore pageWriteStore, int pageSizeThreshold, int initialSizePerCol, boolean enableDictionary) {
     super();
     this.pageWriteStore = pageWriteStore;
     this.pageSizeThreshold = pageSizeThreshold;
+    this.initialSizePerCol = initialSizePerCol;
+    this.enableDictionary = enableDictionary;
   }
 
   public ColumnWriter getColumnWriter(ColumnDescriptor path) {
@@ -51,7 +55,7 @@
 
   private ColumnWriterImpl newMemColumn(ColumnDescriptor path) {
     PageWriter pageWriter = pageWriteStore.getPageWriter(path);
-    return new ColumnWriterImpl(path, pageWriter, pageSizeThreshold);
+    return new ColumnWriterImpl(path, pageWriter, pageSizeThreshold, initialSizePerCol, enableDictionary);
   }
 
   @Override
@@ -100,4 +104,14 @@
     }
   }
 
+  public String memUsageString() {
+    StringBuilder b = new StringBuilder("Store {\n");
+    Collection<ColumnWriterImpl> values = columns.values();
+    for (ColumnWriterImpl memColumn : values) {
+      b.append(memColumn.memUsageString(" "));
+    }
+    b.append("}\n");
+    return b.toString();
+  }
+
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java b/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java
index 7643d2c..b4a64aa 100644
--- a/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java
+++ b/parquet-column/src/main/java/parquet/column/impl/ColumnWriterImpl.java
@@ -15,24 +15,36 @@
  */
 package parquet.column.impl;
 
+import static parquet.bytes.BytesInput.concat;
+
 import java.io.IOException;
 
 import parquet.Log;
-import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
 import parquet.column.ColumnDescriptor;
 import parquet.column.ColumnWriter;
+import parquet.column.page.DictionaryPage;
 import parquet.column.page.PageWriter;
 import parquet.column.values.ValuesWriter;
-import parquet.column.values.bitpacking.BitPackingValuesWriter;
+import parquet.column.values.boundedint.DevNullValuesWriter;
+import parquet.column.values.dictionary.DictionaryValuesWriter;
 import parquet.column.values.plain.BooleanPlainValuesWriter;
 import parquet.column.values.plain.PlainValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
 import parquet.io.ParquetEncodingException;
 import parquet.io.api.Binary;
 
-
+/**
+ * Writes (repetition level, definition level, value) triplets and deals with writing pages to the underlying layer.
+ *
+ * @author Julien Le Dem
+ *
+ */
 final class ColumnWriterImpl implements ColumnWriter {
   private static final Log LOG = Log.getLog(ColumnWriterImpl.class);
   private static final boolean DEBUG = false; //Log.DEBUG;
+  private static final int INITIAL_COUNT_FOR_SIZE_CHECK = 100;
+  private static final int DICTIONARY_PAGE_MAX_SIZE_PERCENT = 20;
 
   private final ColumnDescriptor path;
   private final PageWriter pageWriter;
@@ -41,33 +53,82 @@
   private ValuesWriter definitionLevelColumn;
   private ValuesWriter dataColumn;
   private int valueCount;
+  private int valueCountForNextSizeCheck;
 
-  public ColumnWriterImpl(ColumnDescriptor path, PageWriter pageWriter, int pageSizeThreshold) {
+  public ColumnWriterImpl(ColumnDescriptor path, PageWriter pageWriter, int pageSizeThreshold, int initialSizePerCol, boolean enableDictionary) {
     this.path = path;
     this.pageWriter = pageWriter;
     this.pageSizeThreshold = pageSizeThreshold;
-    repetitionLevelColumn = new BitPackingValuesWriter(path.getMaxRepetitionLevel());
-    definitionLevelColumn = new BitPackingValuesWriter(path.getMaxDefinitionLevel());
+    // initial check of memory usage. So that we have enough data to make an initial prediction
+    this.valueCountForNextSizeCheck = INITIAL_COUNT_FOR_SIZE_CHECK;
+
+    repetitionLevelColumn = getColumnDescriptorValuesWriter(path.getMaxRepetitionLevel());
+
+    definitionLevelColumn = getColumnDescriptorValuesWriter(path.getMaxDefinitionLevel());
+
     switch (path.getType()) {
     case BOOLEAN:
-      this.dataColumn = new BooleanPlainValuesWriter(pageSizeThreshold * 11 / 10);
+      this.dataColumn = new BooleanPlainValuesWriter();
+      break;
+    case BINARY:
+      if (enableDictionary) {
+        this.dataColumn = new DictionaryValuesWriter(applyRatioInPercent(pageSizeThreshold, DICTIONARY_PAGE_MAX_SIZE_PERCENT), initialSizePerCol);
+      } else {
+        this.dataColumn = new PlainValuesWriter(initialSizePerCol);
+      }
       break;
     default:
-      this.dataColumn = new PlainValuesWriter(pageSizeThreshold * 11 / 10);
+      this.dataColumn = new PlainValuesWriter(initialSizePerCol);
     }
   }
 
+  private ValuesWriter getColumnDescriptorValuesWriter(int maxLevel) {
+    if(maxLevel == 0) {
+      return new DevNullValuesWriter();
+    }
+    else {
+      // TODO: what is a good initialCapacity?
+      return new RunLengthBitPackingHybridValuesWriter(
+        BytesUtils.getWidthFromMaxInt(maxLevel),
+        64 * 1024);
+    }
+  }
+
+  private int applyRatioInPercent(int value, int ratio) {
+    if (100 % ratio != 0) {
+      throw new IllegalArgumentException("ratio should be a diviser of 100: not " + ratio);
+    }
+    return value / (100 / ratio);
+  }
+
   private void log(Object value, int r, int d) {
     LOG.debug(path+" "+value+" r:"+r+" d:"+d);
   }
 
+  /**
+   * Counts how many values have been written and checks the memory usage to flush the page when we reach the page threshold.
+   *
+   * We measure the memory used when we reach the mid point toward our estimated count.
+   * We then update the estimate and flush the page if we reached the threshold.
+   *
+   * That way we check the memory size log2(n) times.
+   *
+   */
   private void accountForValueWritten() {
     ++ valueCount;
-    long memSize = repetitionLevelColumn.getBufferedSize()
-        + definitionLevelColumn.getBufferedSize()
-        + dataColumn.getBufferedSize();
-    if (memSize > pageSizeThreshold) {
-      writePage();
+    if (valueCount > valueCountForNextSizeCheck) {
+      // not checking the memory used for every value
+      long memSize = repetitionLevelColumn.getBufferedSize()
+          + definitionLevelColumn.getBufferedSize()
+          + dataColumn.getBufferedSize();
+      if (memSize > pageSizeThreshold) {
+        // we will write the current page and check again the size at the predicted middle of next page
+        valueCountForNextSizeCheck = valueCount / 2;
+        writePage();
+      } else {
+        // not reached the threshold, will check again midway
+        valueCountForNextSizeCheck = (int)(valueCount + ((float)valueCount * pageSizeThreshold / memSize)) / 2 + 1;
+      }
     }
   }
 
@@ -75,7 +136,7 @@
     if (DEBUG) LOG.debug("write page");
     try {
       pageWriter.writePage(
-          BytesInput.fromSequence(repetitionLevelColumn.getBytes(), definitionLevelColumn.getBytes(), dataColumn.getBytes()),
+          concat(repetitionLevelColumn.getBytes(), definitionLevelColumn.getBytes(), dataColumn.getBytes()),
           valueCount,
           repetitionLevelColumn.getEncoding(),
           definitionLevelColumn.getEncoding(),
@@ -156,6 +217,16 @@
     if (valueCount > 0) {
       writePage();
     }
+    final DictionaryPage dictionaryPage = dataColumn.createDictionaryPage();
+    if (dictionaryPage != null) {
+      if (DEBUG) LOG.debug("write dictionary");
+      try {
+        pageWriter.writeDictionaryPage(dictionaryPage);
+      } catch (IOException e) {
+        throw new ParquetEncodingException("could not write dictionary page for " + path, e);
+      }
+      dataColumn.resetDictionary();
+    }
   }
 
   @Override
@@ -172,4 +243,15 @@
     + dataColumn.getAllocatedSize()
     + pageWriter.allocatedSize();
   }
-}
\ No newline at end of file
+
+  public String memUsageString(String indent) {
+    StringBuilder b = new StringBuilder(indent).append(path).append(" {\n");
+    b.append(repetitionLevelColumn.memUsageString(indent + "  r:")).append("\n");
+    b.append(definitionLevelColumn.memUsageString(indent + "  d:")).append("\n");
+    b.append(dataColumn.memUsageString(indent + "  data:")).append("\n");
+    b.append(pageWriter.memUsageString(indent + "  pages:")).append("\n");
+    b.append(indent).append(String.format("  total: %,d/%,d", getBufferedSizeInMemory(), allocatedSize())).append("\n");
+    b.append(indent).append("}\n");
+    return b.toString();
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/page/DictionaryPage.java b/parquet-column/src/main/java/parquet/column/page/DictionaryPage.java
new file mode 100644
index 0000000..78e88a2
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/page/DictionaryPage.java
@@ -0,0 +1,89 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.page;
+
+import static parquet.Preconditions.checkNotNull;
+
+import java.io.IOException;
+
+import parquet.bytes.BytesInput;
+import parquet.column.Encoding;
+
+/**
+ * Data for a dictionary page
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class DictionaryPage {
+
+  private final BytesInput bytes;
+  private final int uncompressedSize;
+  private final int dictionarySize;
+  private final Encoding encoding;
+
+  /**
+   * creates an uncompressed page
+   * @param bytes the content of the page
+   * @param dictionarySize the value count in the dictionary
+   * @param encoding the encoding used
+   */
+  public DictionaryPage(BytesInput bytes, int dictionarySize, Encoding encoding) {
+    this(bytes, (int)bytes.size(), dictionarySize, encoding); // TODO: fix sizes long or int
+  }
+
+  /**
+   * creates a dictionary page
+   * @param bytes the (possibly compressed) content of the page
+   * @param uncompressedSize the size uncompressed
+   * @param dictionarySize the value count in the dictionary
+   * @param encoding the encoding used
+   */
+  public DictionaryPage(BytesInput bytes, int uncompressedSize, int dictionarySize, Encoding encoding) {
+    this.bytes = checkNotNull(bytes, "bytes");
+    this.uncompressedSize = uncompressedSize;
+    this.dictionarySize = dictionarySize;
+    this.encoding = checkNotNull(encoding, "encoding");
+  }
+
+  public BytesInput getBytes() {
+    return bytes;
+  }
+
+  public int getUncompressedSize() {
+    return uncompressedSize;
+  }
+
+  public int getDictionarySize() {
+    return dictionarySize;
+  }
+
+  public Encoding getEncoding() {
+    return encoding;
+  }
+
+  public DictionaryPage copy() throws IOException {
+    return new DictionaryPage(BytesInput.copy(bytes), uncompressedSize, dictionarySize, encoding);
+  }
+
+
+  @Override
+  public String toString() {
+    return "Page [bytes.size=" + bytes.size() + ", entryCount=" + dictionarySize + ", uncompressedSize=" + uncompressedSize + ", encoding=" + encoding + "]";
+  }
+
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/page/PageReader.java b/parquet-column/src/main/java/parquet/column/page/PageReader.java
index 6121221..f2ef171 100644
--- a/parquet-column/src/main/java/parquet/column/page/PageReader.java
+++ b/parquet-column/src/main/java/parquet/column/page/PageReader.java
@@ -23,15 +23,18 @@
  */
 public interface PageReader {
 
-  /**
-   *
-   * @return the total number of values in the column chunk
-   */
-  abstract public long getTotalValueCount();
+ /**
+  * @return the dictionary page in that chunk or null if none
+  */
+  DictionaryPage readDictionaryPage();
 
   /**
-   *
-   * @return the next page in that chunk
+   * @return the total number of values in the column chunk
    */
-  abstract public Page readPage();
+  long getTotalValueCount();
+
+  /**
+   * @return the next page in that chunk or null if after the last page
+   */
+  Page readPage();
 }
diff --git a/parquet-column/src/main/java/parquet/column/page/PageWriter.java b/parquet-column/src/main/java/parquet/column/page/PageWriter.java
index d6df4e9..a3b3b09 100644
--- a/parquet-column/src/main/java/parquet/column/page/PageWriter.java
+++ b/parquet-column/src/main/java/parquet/column/page/PageWriter.java
@@ -37,18 +37,24 @@
    * @param valuesEncoding values encoding
    * @throws IOException
    */
-  abstract public void writePage(BytesInput bytesInput, int valueCount, Encoding rlEncoding, Encoding dlEncoding, Encoding valuesEncoding) throws IOException;
+  void writePage(BytesInput bytesInput, int valueCount, Encoding rlEncoding, Encoding dlEncoding, Encoding valuesEncoding) throws IOException;
 
   /**
-   *
    * @return the current size used in the memory buffer for that column chunk
    */
-  abstract public long getMemSize();
+  long getMemSize();
 
   /**
-   *
    * @return the allocated size for the buffer ( > getMemSize() )
    */
-  public abstract long allocatedSize();
+  long allocatedSize();
+
+  /**
+   * writes a dictionary page
+   * @param dictionaryPage the dictionary page containing the dictionary data
+   */
+  void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException;
+
+  public abstract String memUsageString(String prefix);
 
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesReader.java b/parquet-column/src/main/java/parquet/column/values/ValuesReader.java
index fd0a3ce..57a2bd3 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/ValuesReader.java
@@ -22,10 +22,10 @@
 /**
  * Base class to implement an encoding for a given column type.
  *
- * A PrimitiveColumnReader is provided with a page (byte-array) and is responsible
+ * A ValuesReader is provided with a page (byte-array) and is responsible
  * for deserializing the primitive values stored in that page.
- * 
- * Given that pages are homogenous (store only a single type), typical subclasses
+ *
+ * Given that pages are homogeneous (store only a single type), typical subclasses
  * will only override one of the read*() methods.
  *
  * @author Julien Le Dem
@@ -37,14 +37,14 @@
    *
    * The underlying implementation knows how much data to read, so a length
    * is not provided.
-   * 
+   *
    * Each page may contain several sections:
    * <ul>
    *  <li> repetition levels column
    *  <li> definition levels column
    *  <li> data column
    * </ul>
-   * 
+   *
    * This function is called with 'offset' pointing to the beginning of one of these sections,
    * and should return the offset to the section following it.
    *
@@ -57,6 +57,14 @@
   public abstract int initFromPage(long valueCount, byte[] page, int offset) throws IOException;
 
   /**
+   * usable when the encoding is dictionary based
+   * @return the id of the next value from the page
+   */
+  public int readValueDictionaryId() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
    * @return the next boolean from the page
    */
   public boolean readBoolean() {
@@ -64,44 +72,42 @@
   }
 
   /**
-   * @return the next boolean from the page
-   */
-  public int readByte() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * @return the next boolean from the page
-   */
-  public float readFloat() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * @return the next boolean from the page
+   * @return the next Binary from the page
    */
   public Binary readBytes() {
     throw new UnsupportedOperationException();
   }
 
   /**
-   * @return the next boolean from the page
+   * @return the next float from the page
+   */
+  public float readFloat() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * @return the next double from the page
    */
   public double readDouble() {
     throw new UnsupportedOperationException();
   }
 
   /**
-   * @return the next boolean from the page
+   * @return the next integer from the page
    */
   public int readInteger() {
     throw new UnsupportedOperationException();
   }
 
   /**
-   * @return the next boolean from the page
+   * @return the next long from the page
    */
   public long readLong() {
     throw new UnsupportedOperationException();
   }
+
+  /**
+   * Skips the next value in the page
+   */
+  abstract public void skip();
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/ValuesWriter.java
index 0ed51cf..a0c949c 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/ValuesWriter.java
@@ -17,6 +17,7 @@
 
 import parquet.bytes.BytesInput;
 import parquet.column.Encoding;
+import parquet.column.page.DictionaryPage;
 import parquet.io.api.Binary;
 
 /**
@@ -29,12 +30,13 @@
 
   /**
    * used to decide if we want to work to the next page
-   * @return the size of the currently buffered data
+   * @return the size of the currently buffered data (in bytes)
    */
   public abstract long getBufferedSize();
 
+
+  // TODO: maybe consolidate into a getPage
   /**
-   *
    * @return the bytes buffered so far to write to the current page
    */
   public abstract BytesInput getBytes();
@@ -51,6 +53,19 @@
   public abstract void reset();
 
   /**
+   * @return the dictionary page or null if not dictionary based
+   */
+  public DictionaryPage createDictionaryPage() {
+    return null;
+  }
+
+  /**
+   * reset the dictionary when a new block starts
+   */
+  public void resetDictionary() {
+  }
+
+  /**
    *
    * @return the allocated size of the buffer
    * ( > {@link #getBufferedMemorySize()() )
@@ -106,4 +121,6 @@
     throw new UnsupportedOperationException(getClass().getName());
   }
 
+  abstract public String memUsageString(String prefix);
+
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesReader.java b/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesReader.java
index 6f8f6f1..17c21bc 100644
--- a/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesReader.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 
 import parquet.Log;
+import parquet.bytes.BytesUtils;
 import parquet.column.values.ValuesReader;
 import parquet.column.values.bitpacking.BitPacking.BitPackingReader;
 import parquet.io.ParquetDecodingException;
@@ -40,7 +41,6 @@
   private final int bitsPerValue;
 
   /**
-   *
    * @param bound the maximum value stored by this column
    */
   public BitPackingValuesReader(int bound) {
@@ -48,7 +48,6 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.values.ValuesReader#readInteger()
    */
@@ -62,7 +61,6 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.values.ValuesReader#initFromPage(long, byte[], int)
    */
@@ -70,12 +68,16 @@
   public int initFromPage(long valueCount, byte[] in, int offset) throws IOException {
     // TODO: int vs long
     int effectiveBitLength = (int)valueCount * bitsPerValue;
-    // TODO: maybe ((effectiveBitLength - 1) / 8 + 1) here? has fewer conditionals and divides
-    int length = effectiveBitLength / 8 + (effectiveBitLength % 8 == 0 ? 0 : 1); // ceil
+    int length = BytesUtils.paddedByteCountFromBits(effectiveBitLength);
     if (Log.DEBUG) LOG.debug("reading " + length + " bytes for " + valueCount + " values of size " + bitsPerValue + " bits." );
     this.in = new ByteArrayInputStream(in, offset, length);
     this.bitPackingReader = createBitPackingReader(bitsPerValue, this.in, valueCount);
     return offset + length;
   }
 
+  @Override
+  public void skip() {
+    readInteger();
+  }
+
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesWriter.java
index 50818df..52e6514 100644
--- a/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPackingValuesWriter.java
@@ -41,12 +41,11 @@
   private int bitsPerValue;
 
   /**
-   *
    * @param bound the maximum value stored by this column
    */
-  public BitPackingValuesWriter(int bound) {
+  public BitPackingValuesWriter(int bound, int initialCapacity) {
     this.bitsPerValue = getWidthFromMaxInt(bound);
-    this.out = new CapacityByteArrayOutputStream(32*1024); // size needed could be small but starting at 32 is really small
+    this.out = new CapacityByteArrayOutputStream(initialCapacity);
     init();
   }
 
@@ -55,7 +54,6 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.values.ValuesWriter#writeInteger(int)
    */
@@ -69,7 +67,6 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.values.ValuesWriter#getBufferedSize()
    */
@@ -79,7 +76,6 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.values.ValuesWriter#getBytes()
    */
@@ -94,7 +90,6 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.values.ValuesWriter#reset()
    */
@@ -105,7 +100,6 @@
   }
 
   /**
-   *
    * {@inheritDoc}
    * @see parquet.column.values.ValuesWriter#getAllocatedSize()
    */
@@ -115,9 +109,13 @@
   }
 
   @Override
+  public String memUsageString(String prefix) {
+    return out.memUsageString(prefix);
+  }
+
+  @Override
   public Encoding getEncoding() {
     return BIT_PACKED;
   }
 
-
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java b/parquet-column/src/main/java/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java
new file mode 100644
index 0000000..cfd4b7b
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java
@@ -0,0 +1,75 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import parquet.Log;
+import parquet.bytes.BytesUtils;
+import parquet.column.values.ValuesReader;
+
+public class ByteBitPackingValuesReader extends ValuesReader {
+  private static final int VALUES_AT_A_TIME = 8; // because we're using unpack8Values()
+
+  private static final Log LOG = Log.getLog(ByteBitPackingValuesReader.class);
+
+  private final int bitWidth;
+  private final BytePacker packer;
+  private final int[] decoded = new int[VALUES_AT_A_TIME];
+  private int decodedPosition = VALUES_AT_A_TIME - 1;
+  private byte[] encoded;
+  private int encodedPos;
+
+  public ByteBitPackingValuesReader(int bound, Packer packer) {
+    this.bitWidth = BytesUtils.getWidthFromMaxInt(bound);
+    this.packer = packer.newBytePacker(bitWidth);
+  }
+
+  @Override
+  public int readInteger() {
+    ++ decodedPosition;
+    if (decodedPosition == decoded.length) {
+      if (encodedPos + bitWidth > encoded.length) {
+        packer.unpack8Values(Arrays.copyOfRange(encoded, encodedPos, encodedPos + bitWidth), 0, decoded, 0);
+      } else {
+        packer.unpack8Values(encoded, encodedPos, decoded, 0);
+      }
+      encodedPos += bitWidth;
+      decodedPosition = 0;
+    }
+    return decoded[decodedPosition];
+  }
+
+  @Override
+  public int initFromPage(long valueCount, byte[] page, int offset)
+      throws IOException {
+    // TODO: int vs long
+    int effectiveBitLength = (int)valueCount * bitWidth;
+    int length = BytesUtils.paddedByteCountFromBits(effectiveBitLength); // ceil
+    if (Log.DEBUG) LOG.debug("reading " + length + " bytes for " + valueCount + " values of size " + bitWidth + " bits." );
+    this.encoded = page;
+    this.encodedPos = offset;
+    this.decodedPosition = VALUES_AT_A_TIME - 1;
+    return offset + length;
+  }
+
+  @Override
+  public void skip() {
+    readInteger();
+  }
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java
new file mode 100644
index 0000000..cf308da
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java
@@ -0,0 +1,83 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+import static parquet.column.Encoding.BIT_PACKED;
+
+import java.io.IOException;
+
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+import parquet.column.Encoding;
+import parquet.column.values.ValuesWriter;
+import parquet.io.ParquetEncodingException;
+
+public class ByteBitPackingValuesWriter extends ValuesWriter {
+
+  private final Packer packer;
+  private final int bitWidth;
+  private ByteBasedBitPackingEncoder encoder;
+
+  public ByteBitPackingValuesWriter(int bound, Packer packer) {
+    this.packer = packer;
+    this.bitWidth = BytesUtils.getWidthFromMaxInt(bound);
+    this.encoder = new ByteBasedBitPackingEncoder(bitWidth, packer);
+  }
+
+  @Override
+  public void writeInteger(int v) {
+    try {
+      this.encoder.writeInt(v);
+    } catch (IOException e) {
+      throw new ParquetEncodingException(e);
+    }
+  }
+
+  @Override
+  public Encoding getEncoding() {
+    return BIT_PACKED;
+  }
+
+  @Override
+  public BytesInput getBytes() {
+    try {
+      return encoder.toBytes();
+    } catch (IOException e) {
+      throw new ParquetEncodingException(e);
+    }
+  }
+
+  @Override
+  public void reset() {
+    encoder = new ByteBasedBitPackingEncoder(bitWidth, packer);
+  }
+
+  @Override
+  public long getBufferedSize() {
+    return encoder.getBufferSize();
+  }
+
+  @Override
+  public long getAllocatedSize() {
+    return encoder.getAllocatedSize();
+  }
+
+  @Override
+  public String memUsageString(String prefix) {
+    return encoder.memUsageString(prefix);
+  }
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/boundedint/BitWriter.java b/parquet-column/src/main/java/parquet/column/values/boundedint/BitWriter.java
index 816d6fe..cce73d4 100644
--- a/parquet-column/src/main/java/parquet/column/values/boundedint/BitWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/boundedint/BitWriter.java
@@ -16,13 +16,14 @@
 package parquet.column.values.boundedint;
 
 import parquet.Log;
+import parquet.bytes.BytesInput;
 import parquet.bytes.CapacityByteArrayOutputStream;
 
 class BitWriter {
   private static final Log LOG = Log.getLog(BitWriter.class);
   private static final boolean DEBUG = false;//Log.DEBUG;
 
-  private CapacityByteArrayOutputStream baos = new CapacityByteArrayOutputStream(32 * 1024); // default of 32 is too small
+  private CapacityByteArrayOutputStream baos;
   private int currentByte = 0;
   private int currentBytePosition = 0;
   private static final int[] byteToTrueMask = new int[8];
@@ -37,6 +38,10 @@
     }
   }
 
+  public BitWriter(int initialCapacity) {
+    this.baos = new CapacityByteArrayOutputStream(initialCapacity);
+  }
+
   public void writeBit(boolean bit) {
     if (DEBUG) LOG.debug("writing: " + (bit ? "1" : "0"));
     currentByte = setBytePosition(currentByte, currentBytePosition++, bit);
@@ -89,16 +94,15 @@
     return toBinary(val, 8);
   }
 
-  public byte[] finish() {
+  public BytesInput finish() {
     if (!finished) {
       if (currentBytePosition > 0) {
         baos.write(currentByte);
         if (DEBUG) LOG.debug("to buffer: " + toBinary(currentByte));
       }
     }
-    byte[] buf = baos.toByteArray();
     finished = true;
-    return buf;
+    return BytesInput.from(baos);
   }
 
   public void reset() {
@@ -139,10 +143,14 @@
     // the size of baos:
     //   count : 4 bytes (rounded to 8)
     //   buf : 12 bytes (8 ptr + 4 length) should technically be rounded to 8 depending on buffer size
-    return 32 + baos.size();
+    return 32 + (int)baos.size();
   }
 
   public int getCapacity() {
     return baos.getCapacity();
   }
+
+  public String memUsageString(String prefix) {
+    return baos.memUsageString(prefix);
+  }
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesFactory.java b/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesFactory.java
index e062572..a60c481 100644
--- a/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesFactory.java
+++ b/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesFactory.java
@@ -23,7 +23,7 @@
     return bound == 0 ? new ZeroIntegerValuesReader() : new BoundedIntValuesReader(bound);
   }
 
-  public static ValuesWriter getBoundedWriter(int bound) {
-    return bound == 0 ? new DevNullValuesWriter() : new BoundedIntValuesWriter(bound);
+  public static ValuesWriter getBoundedWriter(int bound, int initialCapacity) {
+    return bound == 0 ? new DevNullValuesWriter() : new BoundedIntValuesWriter(bound, initialCapacity);
   }
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesReader.java b/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesReader.java
index 2b4d1b0..105b3e6 100644
--- a/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesReader.java
@@ -76,4 +76,9 @@
     if (DEBUG) LOG.debug("will read next from " + (offset + totalBytes + 4));
     return offset + totalBytes + 4;
   }
+
+  @Override
+  public void skip() {
+    readInteger();
+  }
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesWriter.java
index 9827e8f..7f4ac62 100644
--- a/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/boundedint/BoundedIntValuesWriter.java
@@ -15,6 +15,7 @@
  */
 package parquet.column.values.boundedint;
 
+import static parquet.bytes.BytesInput.concat;
 import static parquet.column.Encoding.RLE;
 import parquet.Log;
 import parquet.bytes.BytesInput;
@@ -43,7 +44,7 @@
   private boolean thereIsABufferedValue = false;
   private int shouldRepeatThreshold = 0;
   private int bitsPerValue;
-  private BitWriter bitWriter = new BitWriter();
+  private BitWriter bitWriter;
   private boolean isFirst = true;
 
   private static final int[] byteToTrueMask = new int[8];
@@ -55,10 +56,11 @@
     }
   }
 
-  public BoundedIntValuesWriter(int bound) {
+  public BoundedIntValuesWriter(int bound, int initialCapacity) {
     if (bound == 0) {
       throw new ParquetEncodingException("Value bound cannot be 0. Use DevNullColumnWriter instead.");
     }
+    this.bitWriter = new BitWriter(initialCapacity);
     bitsPerValue = (int)Math.ceil(Math.log(bound + 1)/Math.log(2));
     shouldRepeatThreshold = (bitsPerValue + 9)/(1 + bitsPerValue);
     if (Log.DEBUG) LOG.debug("init column with bit width of " + bitsPerValue + " and repeat threshold of " + shouldRepeatThreshold);
@@ -77,12 +79,12 @@
   @Override
   public BytesInput getBytes() {
     serializeCurrentValue();
-    byte[] buf = bitWriter.finish();
-    if (Log.DEBUG) LOG.debug("writing a buffer of size " + buf.length + " + 4 bytes");
+    BytesInput buf = bitWriter.finish();
+    if (Log.DEBUG) LOG.debug("writing a buffer of size " + buf.size() + " + 4 bytes");
     // We serialize the length so that on deserialization we can
     // deserialize as we go, instead of having to load everything
     // into memory
-    return BytesInput.fromSequence(BytesInput.fromInt(buf.length), BytesInput.from(buf));
+    return concat(BytesInput.fromInt((int)buf.size()), buf);
   }
 
   @Override
@@ -146,4 +148,9 @@
     return RLE;
   }
 
+  @Override
+  public String memUsageString(String prefix) {
+    return bitWriter.memUsageString(prefix);
+  }
+
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/values/boundedint/DevNullValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/boundedint/DevNullValuesWriter.java
index 7caf724..fb44edf 100644
--- a/parquet-column/src/main/java/parquet/column/values/boundedint/DevNullValuesWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/boundedint/DevNullValuesWriter.java
@@ -78,4 +78,9 @@
   public Encoding getEncoding() {
     return BIT_PACKED;
   }
+
+  @Override
+  public String memUsageString(String prefix) {
+    return prefix + "0";
+  }
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/values/boundedint/ZeroIntegerValuesReader.java b/parquet-column/src/main/java/parquet/column/values/boundedint/ZeroIntegerValuesReader.java
index 5b71ed2..c1a0b28 100644
--- a/parquet-column/src/main/java/parquet/column/values/boundedint/ZeroIntegerValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/boundedint/ZeroIntegerValuesReader.java
@@ -35,4 +35,8 @@
     return offset;
   }
 
+  @Override
+  public void skip() {
+  }
+
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesReader.java b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesReader.java
new file mode 100644
index 0000000..7929b81
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesReader.java
@@ -0,0 +1,88 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.dictionary;
+
+import static parquet.Log.DEBUG;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import parquet.Log;
+import parquet.bytes.BytesUtils;
+import parquet.column.Dictionary;
+import parquet.column.values.ValuesReader;
+import parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import parquet.io.ParquetDecodingException;
+import parquet.io.api.Binary;
+
+/**
+ * Reads values that have been dictionary encoded
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class DictionaryValuesReader extends ValuesReader {
+  private static final Log LOG = Log.getLog(DictionaryValuesReader.class);
+
+  private InputStream in;
+
+  private Dictionary dictionary;
+
+  private RunLengthBitPackingHybridDecoder decoder;
+
+  public DictionaryValuesReader(Dictionary dictionary) {
+    this.dictionary = dictionary;
+  }
+
+  @Override
+  public int initFromPage(long valueCount, byte[] page, int offset)
+      throws IOException {
+    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (page.length - offset));
+    this.in = new ByteArrayInputStream(page, offset, page.length - offset);
+    int bitWidth = BytesUtils.readIntLittleEndianOnOneByte(in);
+    if (DEBUG) LOG.debug("bit width " + bitWidth);
+    decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);
+    return page.length;
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    try {
+      return decoder.readInt();
+    } catch (IOException e) {
+      throw new ParquetDecodingException(e);
+    }
+  }
+
+  @Override
+  public Binary readBytes() {
+    try {
+      return dictionary.decodeToBinary(decoder.readInt());
+    } catch (IOException e) {
+      throw new ParquetDecodingException(e);
+    }
+  }
+
+  @Override
+  public void skip() {
+    try {
+      decoder.readInt(); // Type does not matter as we are just skipping dictionary keys
+    } catch (IOException e) {
+      throw new ParquetDecodingException(e);
+    }
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesWriter.java
new file mode 100644
index 0000000..807dab0
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/dictionary/DictionaryValuesWriter.java
@@ -0,0 +1,244 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.dictionary;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import parquet.Log;
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+import parquet.bytes.CapacityByteArrayOutputStream;
+import parquet.bytes.LittleEndianDataOutputStream;
+import parquet.column.Encoding;
+import parquet.column.page.DictionaryPage;
+import parquet.column.values.ValuesWriter;
+import parquet.column.values.dictionary.IntList.IntIterator;
+import parquet.column.values.plain.PlainValuesWriter;
+import parquet.column.values.rle.RunLengthBitPackingHybridEncoder;
+import parquet.io.ParquetEncodingException;
+import parquet.io.api.Binary;
+
+import static parquet.Log.DEBUG;
+import static parquet.bytes.BytesInput.concat;
+import static parquet.column.Encoding.PLAIN_DICTIONARY;
+
+/**
+ * Will attempt to encode values using a dictionary and fall back to plain encoding
+ *  if the dictionary gets too big
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class DictionaryValuesWriter extends ValuesWriter {
+  private static final Log LOG = Log.getLog(DictionaryValuesWriter.class);
+
+  private static final int MAX_DICTIONARY_ENTRIES = 65535 /* 2^16 - 1 */;
+
+  /**
+   * maximum size in bytes allowed for the dictionary
+   * will fail over to plain encoding if reached
+   */
+  private final int maxDictionaryByteSize;
+
+  /**
+   * contains the values encoded in plain if the dictionary grows too big
+   */
+  private final PlainValuesWriter plainValuesWriter;
+
+  /**
+   * will become true if the dictionary becomes too big
+   */
+  private boolean dictionaryTooBig;
+
+  /**
+   * current size in bytes the dictionary will take once serialized
+   */
+  private int dictionaryByteSize;
+
+  /**
+   * size in bytes of the dictionary at the end of last dictionary encoded page (in case the current page falls back to PLAIN)
+   */
+  private int lastUsedDictionaryByteSize;
+
+  /**
+   * size in items of the dictionary at the end of last dictionary encoded page (in case the current page falls back to PLAIN)
+   */
+  private int lastUsedDictionarySize;
+
+  /**
+   * dictionary
+   */
+  private Map<Binary, Integer> dict;
+
+  /**
+   * dictionary encoded values
+   */
+  private IntList out = new IntList();
+
+  public DictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
+    this.maxDictionaryByteSize = maxDictionaryByteSize;
+    this.plainValuesWriter = new PlainValuesWriter(initialSize);
+    resetDictionary();
+  }
+
+  @Override
+  public void writeBytes(Binary v) {
+    if (!dictionaryTooBig) {
+      writeBytesUsingDict(v);
+      if (dictionaryByteSize > maxDictionaryByteSize || dict.size() > MAX_DICTIONARY_ENTRIES) {
+        // if the dictionary reaches the max byte size or the values can not be encoded on two bytes anymore.
+        if (DEBUG) LOG.debug("dictionary is now too big, falling back to plain: " + dictionaryByteSize + "B and " + dict.size() + " entries");
+        dictionaryTooBig = true;
+        if (lastUsedDictionarySize == 0) {
+          // if we never used the dictionary
+          // we free dictionary encoded data
+          dict = null;
+          dictionaryByteSize = 0;
+          out = null;
+        }
+      }
+    }
+    // write also to plain encoding if we need to fall back
+    plainValuesWriter.writeBytes(v);
+  }
+
+  /**
+   * will add an entry to the dictionary if the value is new
+   * @param v the value to dictionary encode
+   */
+  private void writeBytesUsingDict(Binary v) {
+    Integer id = dict.get(v);
+    if (id == null) {
+      id = dict.size();
+      dict.put(v, id);
+      // length as int (2 bytes) + actual bytes
+      dictionaryByteSize += 2 + v.length();
+    }
+    out.add(id);
+  }
+
+  @Override
+  public long getBufferedSize() {
+    // size that will be written to a page
+    // not including the dictionary size
+    return dictionaryTooBig ? plainValuesWriter.getBufferedSize() : out.size() * 4;
+  }
+
+  @Override
+  public long getAllocatedSize() {
+    // size used in memory
+    return (out == null ? 0 : out.size() * 4) + dictionaryByteSize + plainValuesWriter.getAllocatedSize();
+  }
+
+  @Override
+  public BytesInput getBytes() {
+    if (!dictionaryTooBig && dict.size() > 0) {
+      // remember size of dictionary when we last wrote a page
+      lastUsedDictionarySize = dict.size();
+      lastUsedDictionaryByteSize = dictionaryByteSize;
+      int maxDicId = dict.size() - 1;
+      if (DEBUG) LOG.debug("max dic id " + maxDicId);
+      int bitWidth = BytesUtils.getWidthFromMaxInt(maxDicId);
+
+      // TODO: what is a good initialCapacity?
+      final RunLengthBitPackingHybridEncoder encoder =
+          new RunLengthBitPackingHybridEncoder(BytesUtils.getWidthFromMaxInt(maxDicId), 64 * 1024);
+      IntIterator iterator = out.iterator();
+      try {
+        while (iterator.hasNext()) {
+          encoder.writeInt(iterator.next());
+        }
+        // encodes the bit width
+        byte[] bytesHeader = new byte[] { (byte)bitWidth };
+        BytesInput rleEncodedBytes = encoder.toBytes();
+        if (DEBUG) LOG.debug("rle encoded bytes " + rleEncodedBytes.size());
+        return concat(BytesInput.from(bytesHeader), rleEncodedBytes);
+      } catch (IOException e) {
+        throw new ParquetEncodingException("could not encode the values", e);
+      }
+    }
+    return plainValuesWriter.getBytes();
+  }
+
+  @Override
+  public Encoding getEncoding() {
+    if (!dictionaryTooBig && dict.size() > 0) {
+      return PLAIN_DICTIONARY;
+    }
+    return plainValuesWriter.getEncoding();
+  }
+
+  @Override
+  public void reset() {
+    if (out != null) {
+      out = new IntList();
+    }
+    plainValuesWriter.reset();
+  }
+
+  @Override
+  public DictionaryPage createDictionaryPage() {
+    if (lastUsedDictionarySize > 0) {
+      // return a dictionary only if we actually used it
+      try {
+        CapacityByteArrayOutputStream dictBuf = new CapacityByteArrayOutputStream(lastUsedDictionaryByteSize);
+        LittleEndianDataOutputStream dictOut = new LittleEndianDataOutputStream(dictBuf);
+        Iterator<Binary> entryIterator = dict.keySet().iterator();
+        // write only the part of the dict that we used
+        for (int i = 0; i < lastUsedDictionarySize; i++) {
+          Binary entry = entryIterator.next();
+          dictOut.writeInt(entry.length());
+          entry.writeTo(dictOut);
+        }
+        return new DictionaryPage(BytesInput.from(dictBuf), lastUsedDictionarySize, PLAIN_DICTIONARY);
+      } catch (IOException e) {
+        throw new ParquetEncodingException("Could not generate dictionary Page", e);
+      }
+    }
+    return plainValuesWriter.createDictionaryPage();
+  }
+
+  @Override
+  public void resetDictionary() {
+    lastUsedDictionaryByteSize = 0;
+    lastUsedDictionarySize = 0;
+    dictionaryByteSize = 0;
+    dictionaryTooBig = false;
+    if (dict == null) {
+      dict = new LinkedHashMap<Binary, Integer>();
+    } else {
+      dict.clear();
+    }
+  }
+
+  public int getDictionaryByteSize() {
+    return dictionaryByteSize;
+  }
+
+  @Override
+  public String memUsageString(String prefix) {
+    return String.format("%s DictionaryValuesWriter{\n%s\n%s\n%s\n%s}\n",
+        prefix,
+        plainValuesWriter.memUsageString(prefix + " plain:"),
+        prefix + " dict:" + dictionaryByteSize,
+        prefix + " values:" + (out.size() * 4),
+        prefix
+        );
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/dictionary/IntList.java b/parquet-column/src/main/java/parquet/column/values/dictionary/IntList.java
new file mode 100644
index 0000000..3be4be5
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/dictionary/IntList.java
@@ -0,0 +1,120 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.dictionary;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An append-only integer list
+ * avoids autoboxing and buffer resizing
+ *
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class IntList {
+
+  private static final int SLAB_SIZE = 64 * 1024;
+
+  /**
+   * to iterate on the content of the list
+   * not an actual iterator to avoid autoboxing
+   *
+   * @author Julien Le Dem
+   *
+   */
+  public static class IntIterator {
+
+    private final int[][] slabs;
+    private int current;
+    private final int count;
+
+    /**
+     * slabs will be iterated in order up to the provided count
+     * as the last slab may not be full
+     * @param slabs contain the ints
+     * @param count total count of ints
+     */
+    public IntIterator(int[][] slabs, int count) {
+      this.slabs = slabs;
+      this.count = count;
+    }
+
+    /**
+     * @return wether there is a next value
+     */
+    public boolean hasNext() {
+      return current < count;
+    }
+
+    /**
+     * @return the next int
+     */
+    public int next() {
+      final int result = slabs[current / SLAB_SIZE][current % SLAB_SIZE];
+      ++ current;
+      return result;
+    }
+
+  }
+
+  private List<int[]> slabs = new ArrayList<int[]>();
+  private int[] currentSlab;
+  private int currentSlabPos;
+
+  /**
+   * construct an empty list
+   */
+  public IntList() {
+    initSlab();
+  }
+
+  private void initSlab() {
+    currentSlab = new int[SLAB_SIZE];
+    currentSlabPos = 0;
+  }
+
+  /**
+   * @param i value to append to the end of the list
+   */
+  public void add(int i) {
+    if (currentSlabPos == currentSlab.length) {
+      slabs.add(currentSlab);
+      initSlab();
+    }
+    currentSlab[currentSlabPos] = i;
+    ++ currentSlabPos;
+  }
+
+  /**
+   * (not an actual Iterable)
+   * @return an IntIterator on the content
+   */
+  public IntIterator iterator() {
+    int[][] itSlabs = slabs.toArray(new int[slabs.size() + 1][]);
+    itSlabs[slabs.size()] = currentSlab;
+    return new IntIterator(itSlabs, SLAB_SIZE * slabs.size() + currentSlabPos);
+  }
+
+  /**
+   * @return the current size of the list
+   */
+  public int size() {
+    return SLAB_SIZE * slabs.size() + currentSlabPos;
+  }
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/dictionary/PlainBinaryDictionary.java b/parquet-column/src/main/java/parquet/column/values/dictionary/PlainBinaryDictionary.java
new file mode 100644
index 0000000..17ed10e
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/dictionary/PlainBinaryDictionary.java
@@ -0,0 +1,82 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.dictionary;
+
+import static parquet.bytes.BytesUtils.readIntLittleEndian;
+import static parquet.column.Encoding.PLAIN_DICTIONARY;
+
+import java.io.IOException;
+
+import parquet.column.Dictionary;
+import parquet.column.page.DictionaryPage;
+import parquet.io.ParquetDecodingException;
+import parquet.io.api.Binary;
+
+/**
+ * a simple implementation of dictionary for Binary data
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class PlainBinaryDictionary extends Dictionary {
+
+  private final Binary[] dictionaryData;
+
+  /**
+   * @param dictionaryPage the PLAIN encoded content of the dictionary
+   * @throws IOException
+   */
+  public PlainBinaryDictionary(DictionaryPage dictionaryPage) throws IOException {
+    super(dictionaryPage.getEncoding());
+    if (dictionaryPage.getEncoding() != PLAIN_DICTIONARY) {
+      throw new ParquetDecodingException("Dictionary encoding not supported: " + dictionaryPage.getEncoding());
+    }
+    final byte[] dictionaryBytes = dictionaryPage.getBytes().toByteArray();
+    dictionaryData = new Binary[dictionaryPage.getDictionarySize()];
+    // dictionary values are stored in order: size (4 bytes LE) followed by {size} bytes
+    int offset = 0;
+    for (int i = 0; i < dictionaryData.length; i++) {
+      int length = readIntLittleEndian(dictionaryBytes, offset);
+      // read the length
+      offset += 4;
+      // wrap the content in a binary
+      dictionaryData[i] = Binary.fromByteArray(dictionaryBytes, offset, length);
+      // increment to the next value
+      offset += length;
+    }
+  }
+
+  @Override
+  public Binary decodeToBinary(int id) {
+    return dictionaryData[id];
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("PlainDictionary {\n");
+    for (int i = 0; i < dictionaryData.length; i++) {
+      Binary element = dictionaryData[i];
+      sb.append(i).append(" => ").append(element.toStringUsingUTF8()).append("\n");
+    }
+    return sb.append("}").toString();
+  }
+
+  @Override
+  public int getMaxId() {
+    return dictionaryData.length - 1;
+  }
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/plain/BinaryPlainValuesReader.java b/parquet-column/src/main/java/parquet/column/values/plain/BinaryPlainValuesReader.java
new file mode 100644
index 0000000..a17d80d
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/plain/BinaryPlainValuesReader.java
@@ -0,0 +1,68 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.plain;
+
+import static parquet.Log.DEBUG;
+
+import java.io.IOException;
+
+import parquet.Log;
+import parquet.bytes.BytesUtils;
+import parquet.column.values.ValuesReader;
+import parquet.io.ParquetDecodingException;
+import parquet.io.api.Binary;
+
+public class BinaryPlainValuesReader extends ValuesReader {
+  private static final Log LOG = Log.getLog(BinaryPlainValuesReader.class);
+  private byte[] in;
+  private int offset;
+
+  @Override
+  public Binary readBytes() {
+    try {
+      int length = BytesUtils.readIntLittleEndian(in, offset);
+      int start = offset + 4;
+      offset = start + length;
+      return Binary.fromByteArray(in, start, length);
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read bytes at offset " + offset, e);
+    } catch (RuntimeException e) {
+      throw new ParquetDecodingException("could not read bytes at offset " + offset, e);
+    }
+  }
+
+  @Override
+  public void skip() {
+    try {
+      int length = BytesUtils.readIntLittleEndian(in, offset);
+      offset += 4 + length;
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not skip bytes at offset " + offset, e);
+    } catch (RuntimeException e) {
+      throw new ParquetDecodingException("could not skip bytes at offset " + offset, e);
+    }
+  }
+
+  @Override
+  public int initFromPage(long valueCount, byte[] in, int offset)
+      throws IOException {
+    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.length - offset));
+    this.in = in;
+    this.offset = offset;
+    return in.length;
+  }
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesReader.java b/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesReader.java
index 1d8df9a..c7a4fb7 100644
--- a/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesReader.java
@@ -16,15 +16,13 @@
 package parquet.column.values.plain;
 
 import static parquet.Log.DEBUG;
-import static parquet.column.values.bitpacking.BitPacking.createBitPackingReader;
+import static parquet.column.values.bitpacking.Packer.LITTLE_ENDIAN;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
 
 import parquet.Log;
 import parquet.column.values.ValuesReader;
-import parquet.column.values.bitpacking.BitPacking.BitPackingReader;
-import parquet.io.ParquetDecodingException;
+import parquet.column.values.bitpacking.ByteBitPackingValuesReader;
 
 /**
  * encodes boolean for the plain encoding: one bit at a time (0 = false)
@@ -35,7 +33,7 @@
 public class BooleanPlainValuesReader extends ValuesReader {
   private static final Log LOG = Log.getLog(BooleanPlainValuesReader.class);
 
-  private BitPackingReader in;
+  private ByteBitPackingValuesReader in = new ByteBitPackingValuesReader(1, LITTLE_ENDIAN);
 
   /**
    *
@@ -44,13 +42,17 @@
    */
   @Override
   public boolean readBoolean() {
-    try {
-      return in.read() == 0 ? false : true;
-    } catch (IOException e) {
-      throw new ParquetDecodingException("never happens", e);
-    }
+    return in.readInteger() == 0 ? false : true;
   }
 
+  /**
+   * {@inheritDoc}
+   * @see parquet.column.values.ValuesReader#skipBoolean()
+   */
+  @Override
+  public void skip() {
+    in.readInteger();
+  }
 
   /**
    * {@inheritDoc}
@@ -59,8 +61,7 @@
   @Override
   public int initFromPage(long valueCount, byte[] in, int offset) throws IOException {
     if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.length - offset));
-    this.in = createBitPackingReader(1, new ByteArrayInputStream(in, offset, in.length - offset), valueCount);
-    return in.length;
+    return this.in.initFromPage(valueCount, in, offset);
   }
 
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesWriter.java
index 5559e82..0e2a65c 100644
--- a/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/plain/BooleanPlainValuesWriter.java
@@ -16,17 +16,11 @@
 package parquet.column.values.plain;
 
 import static parquet.column.Encoding.PLAIN;
-import static parquet.column.values.bitpacking.BitPacking.getBitPackingWriter;
-
-import java.io.IOException;
-
-import parquet.Log;
+import static parquet.column.values.bitpacking.Packer.LITTLE_ENDIAN;
 import parquet.bytes.BytesInput;
-import parquet.bytes.CapacityByteArrayOutputStream;
 import parquet.column.Encoding;
 import parquet.column.values.ValuesWriter;
-import parquet.column.values.bitpacking.BitPacking.BitPackingWriter;
-import parquet.io.ParquetEncodingException;
+import parquet.column.values.bitpacking.ByteBitPackingValuesWriter;
 
 
 /**
@@ -36,50 +30,36 @@
  *
  */
 public class BooleanPlainValuesWriter extends ValuesWriter {
-  private static final Log LOG = Log.getLog(BooleanPlainValuesWriter.class);
 
-  private CapacityByteArrayOutputStream out;
-  private BitPackingWriter bitPackingWriter;
+  private ByteBitPackingValuesWriter bitPackingWriter;
 
-  public BooleanPlainValuesWriter(int initialSize) {
-    out = new CapacityByteArrayOutputStream(initialSize);
-    bitPackingWriter = getBitPackingWriter(1, out);
+  public BooleanPlainValuesWriter() {
+    bitPackingWriter = new ByteBitPackingValuesWriter(1, LITTLE_ENDIAN);
   }
 
   @Override
   public final void writeBoolean(boolean v) {
-    try {
-      bitPackingWriter.write(v ? 1 : 0);
-    } catch (IOException e) {
-      throw new ParquetEncodingException("could not write boolean", e);
-    }
+    bitPackingWriter.writeInteger(v ? 1 : 0);
   }
 
   @Override
   public long getBufferedSize() {
-    return out.size();
+    return bitPackingWriter.getBufferedSize();
   }
 
   @Override
   public BytesInput getBytes() {
-    try {
-      bitPackingWriter.finish();
-    } catch (IOException e) {
-      throw new ParquetEncodingException("could not write page", e);
-    }
-    if (Log.DEBUG) LOG.debug("writing a buffer of size " + out.size());
-    return BytesInput.from(out);
+    return bitPackingWriter.getBytes();
   }
 
   @Override
   public void reset() {
-    out.reset();
-    bitPackingWriter = getBitPackingWriter(1, out);
+    bitPackingWriter.reset();
   }
 
   @Override
   public long getAllocatedSize() {
-    return out.getCapacity();
+    return bitPackingWriter.getAllocatedSize();
   }
 
   @Override
@@ -87,4 +67,9 @@
     return PLAIN;
   }
 
+  @Override
+  public String memUsageString(String prefix) {
+    return bitPackingWriter.memUsageString(prefix);
+  }
+
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesReader.java b/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesReader.java
index e0a5e5d..8b0a191 100644
--- a/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesReader.java
+++ b/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesReader.java
@@ -24,75 +24,17 @@
 import parquet.bytes.LittleEndianDataInputStream;
 import parquet.column.values.ValuesReader;
 import parquet.io.ParquetDecodingException;
-import parquet.io.api.Binary;
 
 /**
- * Plain encoding except for booleans
+ * Plain encoding for float, double, int, long
  *
  * @author Julien Le Dem
  *
  */
-public class PlainValuesReader extends ValuesReader {
+abstract public class PlainValuesReader extends ValuesReader {
   private static final Log LOG = Log.getLog(PlainValuesReader.class);
 
-  private LittleEndianDataInputStream in;
-
-  @Override
-  public float readFloat() {
-    try {
-      return in.readFloat();
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read float", e);
-    }
-  }
-
-  @Override
-  public Binary readBytes() {
-    try {
-      byte[] value = new byte[in.readInt()];
-      in.readFully(value);
-      // TODO: we don't need to read to an array.
-      return Binary.fromByteArray(value);
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read bytes", e);
-    }
-  }
-
-  @Override
-  public double readDouble() {
-    try {
-      return in.readDouble();
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read double", e);
-    }
-  }
-
-  @Override
-  public int readInteger() {
-    try {
-      return in.readInt();
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read int", e);
-    }
-  }
-
-  @Override
-  public long readLong() {
-    try {
-      return in.readLong();
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read long", e);
-    }
-  }
-
-  @Override
-  public int readByte() {
-    try {
-      return in.read();
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read byte", e);
-    }
-  }
+  protected LittleEndianDataInputStream in;
 
   /**
    * {@inheritDoc}
@@ -105,4 +47,89 @@
     return in.length;
   }
 
+  public static class DoublePlainValuesReader extends PlainValuesReader {
+
+  @Override
+  public void skip() {
+    try {
+      in.skipBytes(8);
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not skip double", e);
+    }
+  }
+
+    @Override
+    public double readDouble() {
+      try {
+        return in.readDouble();
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not read double", e);
+      }
+    }
+  }
+
+  public static class FloatPlainValuesReader extends PlainValuesReader {
+
+    @Override
+    public void skip() {
+      try {
+        in.skipBytes(4);
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not skip float", e);
+      }
+    }
+
+    @Override
+    public float readFloat() {
+      try {
+        return in.readFloat();
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not read float", e);
+      }
+    }
+  }
+
+  public static class IntegerPlainValuesReader extends PlainValuesReader {
+
+
+
+    @Override
+    public void skip() {
+      try {
+        in.skipBytes(4);
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not skip int", e);
+      }
+    }
+
+    @Override
+    public int readInteger() {
+      try {
+        return in.readInt();
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not read int", e);
+      }
+    }
+  }
+
+  public static class LongPlainValuesReader extends PlainValuesReader {
+
+    @Override
+    public void skip() {
+      try {
+        in.skipBytes(8);
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not skip long", e);
+      }
+    }
+
+    @Override
+    public long readLong() {
+      try {
+        return in.readLong();
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not read long", e);
+      }
+    }
+  }
 }
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesWriter.java
index c79c4cf..18a98f8 100644
--- a/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesWriter.java
+++ b/parquet-column/src/main/java/parquet/column/values/plain/PlainValuesWriter.java
@@ -133,4 +133,9 @@
     return Encoding.PLAIN;
   }
 
+  @Override
+  public String memUsageString(String prefix) {
+    return arrayOut.memUsageString(prefix + " PLAIN");
+  }
+
 }
diff --git a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
new file mode 100644
index 0000000..a1c7b11
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
@@ -0,0 +1,103 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.rle;
+
+import static parquet.Log.DEBUG;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import parquet.Log;
+import parquet.Preconditions;
+import parquet.bytes.BytesUtils;
+import parquet.column.values.bitpacking.BytePacker;
+import parquet.column.values.bitpacking.Packer;
+import parquet.io.ParquetDecodingException;
+
+/**
+ * Decodes values written in the grammar described in {@link RunLengthBitPackingHybridEncoder}
+ *
+ * @author Julien Le Dem
+ */
+public class RunLengthBitPackingHybridDecoder {
+  private static final Log LOG = Log.getLog(RunLengthBitPackingHybridDecoder.class);
+
+  private static enum MODE { RLE, PACKED }
+
+  private final int bitWidth;
+  private final BytePacker packer;
+  private final InputStream in;
+
+  private MODE mode;
+
+  private int currentCount;
+  private int currentValue;
+  private int[] currentBuffer;
+
+  public RunLengthBitPackingHybridDecoder(int bitWidth, InputStream in) {
+    if (DEBUG) LOG.debug("decoding bitWidth " + bitWidth);
+
+    Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bitWidth;
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    this.in = in;
+  }
+
+  public int readInt() throws IOException {
+    if (currentCount == 0) {
+      readNext();
+    }
+    -- currentCount;
+    int result;
+    switch (mode) {
+    case RLE:
+      result = currentValue;
+      break;
+    case PACKED:
+      result = currentBuffer[currentBuffer.length - 1 - currentCount];
+      break;
+    default:
+      throw new ParquetDecodingException("not a valid mode " + mode);
+    }
+    return result;
+  }
+
+  private void readNext() throws IOException {
+    final int header = BytesUtils.readUnsignedVarInt(in);
+    mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+    switch (mode) {
+    case RLE:
+      currentCount = header >>> 1;
+      if (DEBUG) LOG.debug("reading " + currentCount + " values RLE");
+      currentValue = BytesUtils.readIntLittleEndianPaddedOnBitWidth(in, bitWidth);
+      break;
+    case PACKED:
+      int numGroups = header >>> 1;
+      currentCount = numGroups * 8;
+      if (DEBUG) LOG.debug("reading " + currentCount + " values BIT PACKED");
+      currentBuffer = new int[currentCount]; // TODO: reuse a buffer
+      byte[] bytes = new byte[numGroups * bitWidth];
+      new DataInputStream(in).readFully(bytes);
+      for (int valueIndex = 0, byteIndex = 0; valueIndex < currentCount; valueIndex += 8, byteIndex += bitWidth) {
+        packer.unpack8Values(bytes, byteIndex, currentBuffer, valueIndex);
+      }
+      break;
+    default:
+      throw new ParquetDecodingException("not a valid mode " + mode);
+    }
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java
new file mode 100644
index 0000000..e8b4a8a
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java
@@ -0,0 +1,288 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.rle;
+
+import java.io.IOException;
+
+import parquet.Log;
+import parquet.Preconditions;
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+import parquet.bytes.CapacityByteArrayOutputStream;
+import parquet.column.values.bitpacking.BytePacker;
+import parquet.column.values.bitpacking.Packer;
+
+import static parquet.Log.DEBUG;
+
+/**
+ * Encodes values using a combination of run length encoding and bit packing,
+ * according to the following grammar:
+ *
+ * <pre>
+ * {@code
+ * rle-bit-packed-hybrid: <length> <encoded-data>
+ * length := length of the <encoded-data> in bytes stored as 4 bytes little endian
+ * encoded-data := <run>*
+ * run := <bit-packed-run> | <rle-run>
+ * bit-packed-run := <bit-packed-header> <bit-packed-values>
+ * bit-packed-header := varint-encode(<bit-pack-count> << 1 | 1)
+ * // we always bit-pack a multiple of 8 values at a time, so we only store the number of values / 8
+ * bit-pack-count := (number of values in this run) / 8
+ * bit-packed-values :=  bit packed back to back, from LSB to MSB
+ * rle-run := <rle-header> <repeated-value>
+ * rle-header := varint-encode( (number of times repeated) << 1)
+ * repeated-value := value that is repeated, using a fixed-width of round-up-to-next-byte(bit-width)
+ * }
+ * </pre>
+ * NOTE: this class is only responsible for creating and returning the {@code <encoded-data>}
+ *       portion of the above grammar. The {@code <length>} portion is done by
+ *       {@link RunLengthBitPackingHybridValuesWriter}
+ * <p>
+ * Only supports values >= 0 // TODO: is that ok? Should we make a signed version?
+ *
+ * @author Alex Levenson
+ */
+public class RunLengthBitPackingHybridEncoder {
+  private static final Log LOG = Log.getLog(RunLengthBitPackingHybridEncoder.class);
+
+  private final BytePacker packer;
+
+  private final CapacityByteArrayOutputStream baos;
+
+  /**
+   * The bit width used for bit-packing and for writing
+   * the repeated-value
+   */
+  private final int bitWidth;
+
+  /**
+   * Values that are bit packed 8 at at a time are packed into this
+   * buffer, which is then written to baos
+   */
+  private final byte[] packBuffer;
+
+  /**
+   * Previous value written, used to detect repeated values
+   */
+  private int previousValue;
+
+  /**
+   * We buffer 8 values at a time, and either bit pack them
+   * or discard them after writing a rle-run
+   */
+  private final int[] bufferedValues;
+  private int numBufferedValues;
+
+  /**
+   * How many times a value has been repeated
+   */
+  private int repeatCount;
+
+  /**
+   * How many groups of 8 values have been written
+   * to the current bit-packed-run
+   */
+  private int bitPackedGroupCount;
+
+  /**
+   * A "pointer" to a single byte in baos,
+   * which we use as our bit-packed-header. It's really
+   * the logical index of the byte in baos.
+   *
+   * We are only using one byte for this header,
+   * which limits us to writing 504 values per bit-packed-run.
+   *
+   * MSB must be 0 for varint encoding, LSB must be 1 to signify
+   * that this is a bit-packed-header leaves 6 bits to write the
+   * number of 8-groups -> (2^6 - 1) * 8 = 504
+   */
+  private long bitPackedRunHeaderPointer;
+
+  private boolean toBytesCalled;
+
+  public RunLengthBitPackingHybridEncoder(int bitWidth, int initialCapacity) {
+    if (DEBUG) {
+      LOG.debug(String.format("Encoding: RunLengthBitPackingHybridEncoder with "
+        + "bithWidth: %d initialCapacity %d", bitWidth, initialCapacity));
+    }
+
+    Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
+
+    this.bitWidth = bitWidth;
+    this.baos = new CapacityByteArrayOutputStream(initialCapacity);
+    this.packBuffer = new byte[bitWidth];
+    this.bufferedValues = new int[8];
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    reset(false);
+  }
+
+  private void reset(boolean resetBaos) {
+    if (resetBaos) {
+      this.baos.reset();
+    }
+    this.previousValue = 0;
+    this.numBufferedValues = 0;
+    this.repeatCount = 0;
+    this.bitPackedGroupCount = 0;
+    this.bitPackedRunHeaderPointer = -1;
+    this.toBytesCalled = false;
+  }
+
+  public void writeInt(int value) throws IOException {
+    if (value == previousValue) {
+      // keep track of how many times we've seen this value
+      // consecutively
+      ++repeatCount;
+
+      if (repeatCount >= 8) {
+        // we've seen this at least 8 times, we're
+        // certainly going to write an rle-run,
+        // so just keep on counting repeats for now
+        return;
+      }
+    } else {
+      // This is a new value, check if it signals the end of
+      // an rle-run
+      if (repeatCount >= 8) {
+        // it does! write an rle-run
+        writeRleRun();
+      }
+
+      // this is a new value so we've only seen it once
+      repeatCount = 1;
+      // start tracking this value for repeats
+      previousValue = value;
+    }
+
+    // We have not seen enough repeats to justify an rle-run yet,
+    // so buffer this value in case we decide to write a bit-packed-run
+    bufferedValues[numBufferedValues] = value;
+    ++numBufferedValues;
+
+    if (numBufferedValues == 8) {
+      // we've encountered less than 8 repeated values, so
+      // either start a new bit-packed-run or append to the
+      // current bit-packed-run
+      writeOrAppendBitPackedRun();
+    }
+  }
+
+  private void writeOrAppendBitPackedRun() throws IOException {
+    if (bitPackedGroupCount >= 63) {
+      // we've packed as many values as we can for this run,
+      // end it and start a new one
+      endPreviousBitPackedRun();
+    }
+
+    if (bitPackedRunHeaderPointer == -1) {
+      // this is a new bit-packed-run, allocate a byte for the header
+      // and keep a "pointer" to it so that it can be mutated later
+      baos.write(0); // write a sentinel value
+      bitPackedRunHeaderPointer = baos.getCurrentIndex();
+    }
+
+    packer.pack8Values(bufferedValues, 0, packBuffer, 0);
+    baos.write(packBuffer);
+
+    // empty the buffer, they've all been written
+    numBufferedValues = 0;
+
+    // clear the repeat count, as some repeated values
+    // may have just been bit packed into this run
+    repeatCount = 0;
+
+    ++bitPackedGroupCount;
+  }
+
+  /**
+   * If we are currently writing a bit-packed-run, update the
+   * bit-packed-header and consider this run to be over
+   *
+   * does nothing if we're not currently writing a bit-packed run
+   */
+  private void endPreviousBitPackedRun() {
+    if (bitPackedRunHeaderPointer == -1) {
+      // we're not currently in a bit-packed-run
+      return;
+    }
+
+    // create bit-packed-header, which needs to fit in 1 byte
+    byte bitPackHeader = (byte) ((bitPackedGroupCount << 1) | 1);
+
+    // update this byte
+    baos.setByte(bitPackedRunHeaderPointer, bitPackHeader);
+
+    // mark that this run is over
+    bitPackedRunHeaderPointer = -1;
+
+    // reset the number of groups
+    bitPackedGroupCount = 0;
+  }
+
+  private void writeRleRun() throws IOException {
+    // we may have been working on a bit-packed-run
+    // so close that run if it exists before writing this
+    // rle-run
+    endPreviousBitPackedRun();
+
+    // write the rle-header (lsb of 0 signifies a rle run)
+    BytesUtils.writeUnsignedVarInt(repeatCount << 1, baos);
+    // write the repeated-value
+    BytesUtils.writeIntLittleEndianPaddedOnBitWidth(baos, previousValue, bitWidth);
+
+    // reset the repeat count
+    repeatCount = 0;
+
+    // throw away all the buffered values, they were just repeats and they've been written
+    numBufferedValues = 0;
+  }
+
+  public BytesInput toBytes() throws IOException {
+    Preconditions.checkArgument(!toBytesCalled,
+        "You cannot call toBytes() more than once without calling reset()");
+
+    // write anything that is buffered / queued up for an rle-run
+    if (repeatCount >= 8) {
+      writeRleRun();
+    } else if(numBufferedValues > 0) {
+      for (int i = numBufferedValues; i < 8; i++) {
+        bufferedValues[i] = 0;
+      }
+      writeOrAppendBitPackedRun();
+      endPreviousBitPackedRun();
+    } else {
+      endPreviousBitPackedRun();
+    }
+
+    toBytesCalled = true;
+    return BytesInput.from(baos);
+  }
+
+  /**
+   * Reset this encoder for re-use
+   */
+  public void reset() {
+    reset(true);
+  }
+
+  public long getBufferedSize() {
+    return baos.size();
+  }
+
+  public long getAllocatedSize() {
+    return baos.getCapacity();
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
new file mode 100644
index 0000000..1a6e920
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
@@ -0,0 +1,74 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.rle;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+import parquet.Ints;
+import parquet.bytes.BytesUtils;
+import parquet.column.values.ValuesReader;
+import parquet.io.ParquetDecodingException;
+
+/**
+ * This ValuesReader does all the reading in {@link #initFromPage}
+ * and stores the values in an in memory buffer, which is less than ideal.
+ *
+ * @author Alex Levenson
+ */
+public class RunLengthBitPackingHybridValuesReader extends ValuesReader {
+  private final int bitWidth;
+  private RunLengthBitPackingHybridDecoder decoder;
+
+  public RunLengthBitPackingHybridValuesReader(int bitWidth) {
+    this.bitWidth = bitWidth;
+  }
+
+  @Override
+  public int initFromPage(long valueCountL, byte[] page, int offset) throws IOException {
+    // TODO: we are assuming valueCount < Integer.MAX_VALUE
+    //       we should address this here and elsewhere
+    int valueCount = Ints.checkedCast(valueCountL);
+
+    if (valueCount <= 0) {
+      // readInteger() will never be called,
+      // there is no data to read
+      return offset;
+    }
+
+    ByteArrayInputStream in = new ByteArrayInputStream(page, offset, page.length);
+    int length = BytesUtils.readIntLittleEndian(in);
+
+    decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);
+
+    // 4 is for the length which is stored as 4 bytes little endian
+    return offset + length + 4;
+  }
+
+  @Override
+  public int readInteger() {
+    try {
+      return decoder.readInt();
+    } catch (IOException e) {
+      throw new ParquetDecodingException(e);
+    }
+  }
+
+  @Override
+  public void skip() {
+    readInteger();
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
new file mode 100644
index 0000000..817030c
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
@@ -0,0 +1,86 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.rle;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import parquet.Ints;
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+import parquet.column.Encoding;
+import parquet.column.values.ValuesWriter;
+import parquet.io.ParquetEncodingException;
+
+/**
+ * @author Alex Levenson
+ */
+public class RunLengthBitPackingHybridValuesWriter extends ValuesWriter {
+  private final RunLengthBitPackingHybridEncoder encoder;
+  private final ByteArrayOutputStream length;
+
+  public RunLengthBitPackingHybridValuesWriter(int bitWidth, int initialCapacity) {
+    this.encoder = new RunLengthBitPackingHybridEncoder(bitWidth, initialCapacity);
+    this.length = new ByteArrayOutputStream(4);
+  }
+
+  @Override
+  public void writeInteger(int v) {
+    try {
+      encoder.writeInt(v);
+    } catch (IOException e) {
+      throw new ParquetEncodingException(e);
+    }
+  }
+
+  @Override
+  public long getBufferedSize() {
+    return encoder.getBufferedSize();
+  }
+
+  @Override
+  public long getAllocatedSize() {
+    return encoder.getAllocatedSize();
+  }
+
+  @Override
+  public BytesInput getBytes() {
+    try {
+      // prepend the length of the column
+      BytesInput rle = encoder.toBytes();
+      BytesUtils.writeIntLittleEndian(length, Ints.checkedCast(rle.size()));
+      return BytesInput.concat(BytesInput.from(length.toByteArray()), rle);
+    } catch (IOException e) {
+      throw new ParquetEncodingException(e);
+    }
+  }
+
+  @Override
+  public Encoding getEncoding() {
+    return Encoding.RLE;
+  }
+
+  @Override
+  public void reset() {
+    encoder.reset();
+    length.reset();
+  }
+
+  @Override
+  public String memUsageString(String prefix) {
+    return String.format("%s RunLengthBitPackingHybrid %d bytes", prefix, getAllocatedSize());
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/example/data/simple/BinaryValue.java b/parquet-column/src/main/java/parquet/example/data/simple/BinaryValue.java
index fef72f2..53c05cf 100644
--- a/parquet-column/src/main/java/parquet/example/data/simple/BinaryValue.java
+++ b/parquet-column/src/main/java/parquet/example/data/simple/BinaryValue.java
@@ -15,7 +15,6 @@
  */
 package parquet.example.data.simple;
 
-import parquet.bytes.BytesUtils;
 import parquet.io.api.Binary;
 import parquet.io.api.RecordConsumer;
 
diff --git a/parquet-column/src/main/java/parquet/filter/AndRecordFilter.java b/parquet-column/src/main/java/parquet/filter/AndRecordFilter.java
new file mode 100644
index 0000000..04221de
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/filter/AndRecordFilter.java
@@ -0,0 +1,66 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.filter;
+
+import parquet.Preconditions;
+import parquet.column.ColumnReader;
+
+/**
+ * Provides ability to chain two filters together. Bear in mind that the first one will
+ * short circuit the second. Useful if getting a page of already filtered result.
+ *  i.e and( column("manufacturer", equalTo("Volkswagen")), page(100,50))
+ *
+ * @author Jacob Metcalf
+ */
+public final class AndRecordFilter implements RecordFilter {
+
+  private final RecordFilter boundFilter1;
+  private final RecordFilter boundFilter2;
+
+  /**
+   * Returns builder for creating an and filter.
+   * @param filter1 The first filter to check.
+   * @param filter2 The second filter to check.
+   */
+  public static final UnboundRecordFilter and( final UnboundRecordFilter filter1, final UnboundRecordFilter filter2 ) {
+    Preconditions.checkNotNull( filter1, "filter1" );
+    Preconditions.checkNotNull( filter2, "filter2" );
+    return new UnboundRecordFilter() {
+      @Override
+      public RecordFilter bind(Iterable<ColumnReader> readers) {
+        return new AndRecordFilter( filter1.bind(readers), filter2.bind( readers) );
+      }
+    };
+  }
+
+  /**
+   * Private constructor, use AndRecordFilter.and() instead.
+   */
+  private AndRecordFilter( RecordFilter boundFilter1, RecordFilter boundFilter2 ) {
+    this.boundFilter1 = boundFilter1;
+    this.boundFilter2 = boundFilter2;
+  }
+
+  @Override
+  public boolean isFullyConsumed() {
+    return boundFilter1.isFullyConsumed() && boundFilter2.isFullyConsumed();
+  }
+
+  @Override
+  public boolean isMatch() {
+    return boundFilter1.isMatch() && boundFilter2.isMatch();
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/filter/ColumnPredicates.java b/parquet-column/src/main/java/parquet/filter/ColumnPredicates.java
new file mode 100644
index 0000000..b8fc10b
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/filter/ColumnPredicates.java
@@ -0,0 +1,97 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.filter;
+
+import parquet.Preconditions;
+import parquet.column.ColumnReader;
+
+/**
+ * ColumnPredicates class provides checks for column values. Factory methods
+ * are provided for standard predicates which wrap the job of getting the
+ * correct value from the column.
+ */
+public class ColumnPredicates {
+
+  public static interface Predicate {
+    boolean apply(ColumnReader input);
+  }
+
+  public static Predicate equalTo(final String target) {
+    Preconditions.checkNotNull(target,"target");
+    return new Predicate() {
+      @Override
+      public boolean apply(ColumnReader input) {
+        return target.equals(input.getBinary().toStringUsingUTF8());
+      }
+    };
+  }
+
+  public static Predicate equalTo(final int target) {
+    return new Predicate() {
+      @Override
+      public boolean apply(ColumnReader input) {
+        return input.getInteger() == target;
+      }
+    };
+  }
+
+  public static Predicate equalTo(final long target) {
+    return new Predicate() {
+      @Override
+      public boolean apply(ColumnReader input) {
+        return input.getLong() == target;
+      }
+    };
+  }
+
+  public static Predicate equalTo(final float target) {
+    return new Predicate() {
+      @Override
+      public boolean apply(ColumnReader input) {
+        return input.getFloat() == target;
+      }
+    };
+  }
+
+  public static Predicate equalTo(final double target) {
+    return new Predicate() {
+      @Override
+      public boolean apply(ColumnReader input) {
+        return input.getDouble() == target;
+      }
+    };
+  }
+
+  public static Predicate equalTo(final boolean target) {
+    return new Predicate() {
+      @Override
+      public boolean apply(ColumnReader input) {
+        return input.getBoolean() == target;
+      }
+    };
+  }
+
+  public static <E extends Enum> Predicate equalTo(final E target) {
+    Preconditions.checkNotNull(target,"target");
+    final String targetAsString = target.name();
+    return new Predicate() {
+      @Override
+      public boolean apply(ColumnReader input) {
+        return targetAsString.equals(input.getBinary().toStringUsingUTF8());
+      }
+    };
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/filter/ColumnRecordFilter.java b/parquet-column/src/main/java/parquet/filter/ColumnRecordFilter.java
new file mode 100644
index 0000000..104416f
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/filter/ColumnRecordFilter.java
@@ -0,0 +1,80 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.filter;
+
+import parquet.column.ColumnReader;
+import java.util.Arrays;
+import static parquet.Preconditions.checkNotNull;
+
+/**
+ * Record filter which applies the supplied predicate to the specified column.
+ */
+public final class ColumnRecordFilter implements RecordFilter {
+
+  private final ColumnReader filterOnColumn;
+  private final ColumnPredicates.Predicate filterPredicate;
+
+  /**
+   * Factory method for record filter which applies the supplied predicate to the specified column.
+   * Note that if searching for a repeated sub-attribute it will only ever match against the
+   * first instance of it in the object.
+   *
+   * @param columnPath Dot separated path specifier, e.g. "engine.capacity"
+   * @param predicate Should call getBinary etc. and check the value
+   */
+  public static final UnboundRecordFilter column(final String columnPath,
+                                                 final ColumnPredicates.Predicate predicate) {
+    checkNotNull(columnPath, "columnPath");
+    checkNotNull(predicate,  "predicate");
+    return new UnboundRecordFilter() {
+      final String[] filterPath = columnPath.split("\\.");
+      @Override
+      public RecordFilter bind(Iterable<ColumnReader> readers) {
+        for (ColumnReader reader : readers) {
+          if ( Arrays.equals( reader.getDescriptor().getPath(), filterPath)) {
+            return new ColumnRecordFilter(reader, predicate);
+          }
+        }
+        throw new IllegalArgumentException( "Column " + columnPath + " does not exist.");
+      }
+    };
+  }
+
+  /**
+   * Private constructor. Use column() instead.
+   */
+  private ColumnRecordFilter(ColumnReader filterOnColumn, ColumnPredicates.Predicate filterPredicate) {
+    this.filterOnColumn  = filterOnColumn;
+    this.filterPredicate = filterPredicate;
+  }
+
+  /**
+   * @return true if the current value for the column reader matches the predicate.
+   */
+  @Override
+  public boolean isMatch() {
+
+    return ( filterOnColumn.isFullyConsumed()) ? false : filterPredicate.apply( filterOnColumn );
+  }
+
+  /**
+   * @return true if the column we are filtering on has no more values.
+   */
+  @Override
+  public boolean isFullyConsumed() {
+    return filterOnColumn.isFullyConsumed();
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/filter/PagedRecordFilter.java b/parquet-column/src/main/java/parquet/filter/PagedRecordFilter.java
new file mode 100644
index 0000000..4185112
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/filter/PagedRecordFilter.java
@@ -0,0 +1,69 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.filter;
+
+import parquet.column.ColumnReader;
+
+/**
+ * Filter which will only materialize a page worth of results.
+ */
+public final class PagedRecordFilter implements RecordFilter {
+
+  private final long startPos;
+  private final long endPos;
+  private long currentPos = 0;
+
+  /**
+   * Returns builder for creating a paged query.
+   * @param startPos The record to start from, numbering starts at 1.
+   * @param pageSize The size of the page.
+   */
+  public static final UnboundRecordFilter page( final long startPos, final long pageSize ) {
+    return new UnboundRecordFilter() {
+      @Override
+      public RecordFilter bind(Iterable<ColumnReader> readers) {
+        return new PagedRecordFilter( startPos, pageSize );
+      }
+    };
+  }
+
+  /**
+   * Private constructor, use column() instead.
+   */
+  private PagedRecordFilter(long startPos, long pageSize) {
+    this.startPos = startPos;
+    this.endPos   = startPos + pageSize;
+  }
+
+  /**
+   * Terminate early when we have got our page.
+   */
+  @Override
+  public boolean isFullyConsumed() {
+    return ( currentPos >= endPos );
+  }
+
+  /**
+   * Keeps track of how many times it is called. Only returns matches when the
+   * record number is in the range.
+   */
+  @Override
+  public boolean isMatch() {
+    currentPos++;
+    return (( currentPos >= startPos ) && ( currentPos < endPos ));
+  }
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesType.java b/parquet-column/src/main/java/parquet/filter/RecordFilter.java
similarity index 60%
copy from parquet-column/src/main/java/parquet/column/values/ValuesType.java
copy to parquet-column/src/main/java/parquet/filter/RecordFilter.java
index 0566303..569835e 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesType.java
+++ b/parquet-column/src/main/java/parquet/filter/RecordFilter.java
@@ -13,14 +13,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.values;
+package parquet.filter;
+
+import parquet.column.ColumnReader;
 
 /**
- * The different type of values we can store in columns
+ * Filter to be applied to a record to work out whether to skip it.
  *
- * @author Julien Le Dem
- *
+ * @author Jacob Metcalf
  */
-public enum ValuesType {
-  REPETITION_LEVEL, DEFINITION_LEVEL, VALUES;
+public interface RecordFilter {
+
+  /**
+   * Works out whether the current record can pass through the filter.
+   */
+  boolean isMatch();
+
+  /**
+   * Whether the filter values are fully consumed.
+   */
+  boolean isFullyConsumed();
 }
diff --git a/parquet-column/src/main/java/parquet/filter/UnboundRecordFilter.java b/parquet-column/src/main/java/parquet/filter/UnboundRecordFilter.java
new file mode 100644
index 0000000..b317727
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/filter/UnboundRecordFilter.java
@@ -0,0 +1,33 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.filter;
+
+import parquet.column.ColumnReader;
+
+/**
+ * Builder for a record filter. Idea is that each filter provides a create function
+ * which returns an unbound filter. This only becomes a filter when it is bound to the actual
+ * columns.
+ *
+ * @author Jacob Metcalf
+ */
+public interface UnboundRecordFilter {
+
+  /**
+   * Call to bind to actual columns and create filter.
+   */
+  RecordFilter bind( Iterable<ColumnReader> readers);
+}
diff --git a/parquet-column/src/main/java/parquet/io/ColumnIOFactory.java b/parquet-column/src/main/java/parquet/io/ColumnIOFactory.java
index 87b2817..0c2fab7 100644
--- a/parquet-column/src/main/java/parquet/io/ColumnIOFactory.java
+++ b/parquet-column/src/main/java/parquet/io/ColumnIOFactory.java
@@ -15,6 +15,8 @@
  */
 package parquet.io;
 
+import static parquet.schema.Type.Repetition.REQUIRED;
+
 import java.util.ArrayList;
 import java.util.List;
 
@@ -23,7 +25,6 @@
 import parquet.schema.PrimitiveType;
 import parquet.schema.Type;
 import parquet.schema.TypeVisitor;
-import parquet.schema.Type.Repetition;
 
 /**
  * Factory constructing the ColumnIO structure from the schema
@@ -39,47 +40,68 @@
     private GroupColumnIO current;
     private List<PrimitiveColumnIO> leaves = new ArrayList<PrimitiveColumnIO>();
     private final boolean validating;
+    private final MessageType requestedSchema;
+    private int currentRequestedIndex;
+    private Type currentRequestedType;
 
-    public ColumnIOCreatorVisitor(boolean validating) {
+    public ColumnIOCreatorVisitor(boolean validating, MessageType requestedSchema) {
       this.validating = validating;
-    }
-
-    @Override
-    public void visit(GroupType groupType) {
-      GroupColumnIO newIO;
-      if (groupType.getRepetition() == Repetition.REPEATED) {
-        newIO = new GroupColumnIO(groupType, current, current.getChildrenCount());
-      } else {
-        newIO = new GroupColumnIO(groupType, current, current.getChildrenCount());
-      }
-      current.add(newIO);
-      visitChildren(newIO, groupType);
-    }
-
-    private void visitChildren(GroupColumnIO newIO, GroupType groupType) {
-      GroupColumnIO oldIO = current;
-      current = newIO;
-      for (Type type : groupType.getFields()) {
-        type.accept(this);
-      }
-      current = oldIO;
+      this.requestedSchema = requestedSchema;
     }
 
     @Override
     public void visit(MessageType messageType) {
-      columnIO = new MessageColumnIO(messageType, validating);
-      visitChildren(columnIO, messageType);
+      columnIO = new MessageColumnIO(requestedSchema, validating);
+      visitChildren(columnIO, messageType, requestedSchema);
       columnIO.setLevels();
       columnIO.setLeaves(leaves);
     }
 
     @Override
+    public void visit(GroupType groupType) {
+      if (currentRequestedType.isPrimitive()) {
+        incompatibleSchema(groupType, currentRequestedType);
+      }
+      GroupColumnIO newIO = new GroupColumnIO(groupType, current, currentRequestedIndex);
+      current.add(newIO);
+      visitChildren(newIO, groupType, currentRequestedType.asGroupType());
+    }
+
+    private void visitChildren(GroupColumnIO newIO, GroupType groupType, GroupType requestedGroupType) {
+      GroupColumnIO oldIO = current;
+      current = newIO;
+      for (Type type : groupType.getFields()) {
+        // if the file schema does not contain the field it will just stay null
+        if (requestedGroupType.containsField(type.getName())) {
+          currentRequestedIndex = requestedGroupType.getFieldIndex(type.getName());
+          currentRequestedType = requestedGroupType.getType(currentRequestedIndex);
+          if (currentRequestedType.getRepetition().isMoreRestrictiveThan(type.getRepetition())) {
+            incompatibleSchema(type, currentRequestedType);
+          }
+          type.accept(this);
+        } else if (type.getRepetition() == REQUIRED) {
+          // if the missing field is required we fail
+          // TODO: add support for default values
+          throw new ParquetDecodingException("The requested schema is not compatible with the file schema. Missing required field in file " + type);
+        }
+      }
+      current = oldIO;
+    }
+
+    @Override
     public void visit(PrimitiveType primitiveType) {
-      PrimitiveColumnIO newIO = new PrimitiveColumnIO(primitiveType, current, current.getChildrenCount(), leaves.size());
+      if (!currentRequestedType.isPrimitive() || currentRequestedType.asPrimitiveType().getPrimitiveTypeName() != primitiveType.getPrimitiveTypeName()) {
+        incompatibleSchema(primitiveType, currentRequestedType);
+      }
+      PrimitiveColumnIO newIO = new PrimitiveColumnIO(primitiveType, current, currentRequestedIndex, leaves.size());
       current.add(newIO);
       leaves.add(newIO);
     }
 
+    private void incompatibleSchema(Type fileType, Type requestedType) {
+      throw new ParquetDecodingException("The requested schema is not compatible with the file schema. incompatible types: " + requestedType + " != " + fileType);
+    }
+
     public MessageColumnIO getColumnIO() {
       return columnIO;
     }
@@ -104,13 +126,22 @@
   }
 
   /**
+   * @param schema the requestedSchema we want to read/write
+   * @param fileSchema the file schema (when reading it can be different from the requested schema)
+   * @return the corresponding serializing/deserializing structure
+   */
+  public MessageColumnIO getColumnIO(MessageType requestedSchema, MessageType fileSchema) {
+    ColumnIOCreatorVisitor visitor = new ColumnIOCreatorVisitor(validating, requestedSchema);
+    fileSchema.accept(visitor);
+    return visitor.getColumnIO();
+  }
+
+  /**
    * @param schema the schema we want to read/write
    * @return the corresponding serializing/deserializing structure
    */
   public MessageColumnIO getColumnIO(MessageType schema) {
-    ColumnIOCreatorVisitor visitor = new ColumnIOCreatorVisitor(validating);
-    schema.accept(visitor);
-    return visitor.getColumnIO();
+    return this.getColumnIO(schema, schema);
   }
 
 }
diff --git a/parquet-column/src/main/java/parquet/io/FilteredRecordReader.java b/parquet-column/src/main/java/parquet/io/FilteredRecordReader.java
new file mode 100644
index 0000000..3c1ae92
--- /dev/null
+++ b/parquet-column/src/main/java/parquet/io/FilteredRecordReader.java
@@ -0,0 +1,92 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.io;
+
+import parquet.column.ColumnReader;
+import parquet.column.impl.ColumnReadStoreImpl;
+import parquet.filter.RecordFilter;
+import parquet.filter.UnboundRecordFilter;
+import parquet.io.api.RecordMaterializer;
+
+import java.util.Arrays;
+
+/**
+ * Extends the
+ * @author Jacob Metcalf
+ *
+ */
+class FilteredRecordReader<T> extends RecordReaderImplementation<T> {
+
+  private final RecordFilter recordFilter;
+
+  /**
+   * @param root          the root of the schema
+   * @param validating
+   * @param columnStore
+   * @param unboundFilter Filter records, pass in NULL_FILTER to leave unfiltered.
+   */
+  public FilteredRecordReader(MessageColumnIO root, RecordMaterializer<T> recordMaterializer, boolean validating,
+                              ColumnReadStoreImpl columnStore, UnboundRecordFilter unboundFilter) {
+    super(root, recordMaterializer, validating, columnStore);
+
+    if ( unboundFilter != null ) {
+      recordFilter = unboundFilter.bind(getColumnReaders());
+    } else {
+      recordFilter = null;
+    }
+  }
+
+  /**
+   * Override read() method to provide skip.
+   */
+  @Override
+  public T read() {
+    if ( skipToMatch()) {
+      return super.read();
+    } else {
+      return null;
+    }
+  }
+
+
+  /**
+   * Skips forwards until the filter finds the first match. Returns false
+   * if none found.
+   */
+  private boolean skipToMatch() {
+    while ( !recordFilter.isMatch()) {
+      if ( recordFilter.isFullyConsumed()) {
+        return false;
+      }
+      State currentState = getState(0);
+      do {
+        ColumnReader columnReader = currentState.column;
+
+        // currentLevel = depth + 1 at this point
+        // set the current value
+        if (columnReader.getCurrentDefinitionLevel() >= currentState.maxDefinitionLevel) {
+          columnReader.skip();
+        }
+        columnReader.consume();
+
+        // Based on repetition level work out next state to go to
+        int nextR = currentState.maxRepetitionLevel == 0 ? 0 : columnReader.getCurrentRepetitionLevel();
+        currentState = currentState.getNextState(nextR);
+      } while (currentState != null);
+    }
+    return true;
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/io/GroupColumnIO.java b/parquet-column/src/main/java/parquet/io/GroupColumnIO.java
index 7a87321..aabb0a9 100644
--- a/parquet-column/src/main/java/parquet/io/GroupColumnIO.java
+++ b/parquet-column/src/main/java/parquet/io/GroupColumnIO.java
@@ -57,8 +57,8 @@
     for (ColumnIO child : this.children) {
       String[] newFieldPath = Arrays.copyOf(fieldPath, fieldPath.length + 1);
       int[] newIndexFieldPath = Arrays.copyOf(indexFieldPath, indexFieldPath.length + 1);
-      newFieldPath[fieldPath.length] =  child.getType().getName();
-      newIndexFieldPath[indexFieldPath.length] =  this.getType().asGroupType().getFieldIndex(child.getType().getName());
+      newFieldPath[fieldPath.length] = child.getType().getName();
+      newIndexFieldPath[indexFieldPath.length] = child.getIndex();
       List<ColumnIO> newRepetition;
       if (child.getType().getRepetition() == REPEATED) {
         newRepetition = new ArrayList<ColumnIO>(repetition);
diff --git a/parquet-column/src/main/java/parquet/io/MessageColumnIO.java b/parquet-column/src/main/java/parquet/io/MessageColumnIO.java
index e290143..64c65ad 100644
--- a/parquet-column/src/main/java/parquet/io/MessageColumnIO.java
+++ b/parquet-column/src/main/java/parquet/io/MessageColumnIO.java
@@ -27,6 +27,8 @@
 import parquet.io.api.RecordConsumer;
 import parquet.io.api.RecordMaterializer;
 import parquet.schema.MessageType;
+import parquet.filter.UnboundRecordFilter;
+import parquet.filter.RecordFilter;
 
 /**
  * Message level of the IO structure
@@ -54,7 +56,25 @@
   }
 
   public <T> RecordReader<T> getRecordReader(PageReadStore columns, RecordMaterializer<T> recordMaterializer) {
-    return new RecordReaderImplementation<T>(this, recordMaterializer, validating, new ColumnReadStoreImpl(columns));
+    return new RecordReaderImplementation<T>(
+        this,
+        recordMaterializer,
+        validating,
+        new ColumnReadStoreImpl(columns, recordMaterializer.getRootConverter(), getType())
+    );
+  }
+  public <T> RecordReader<T> getRecordReader(PageReadStore columns, RecordMaterializer<T> recordMaterializer,
+                                             UnboundRecordFilter unboundFilter) {
+
+    return (unboundFilter == null)
+      ? getRecordReader(columns, recordMaterializer)
+      : new FilteredRecordReader<T>(
+        this,
+        recordMaterializer,
+        validating,
+        new ColumnReadStoreImpl(columns, recordMaterializer.getRootConverter(), getType()),
+        unboundFilter
+    );
   }
 
   private class MessageColumnIORecordConsumer extends RecordConsumer {
@@ -63,6 +83,7 @@
     private final int[] currentIndex;
     private final int[] r;
     private final ColumnWriter[] columnWriter;
+    private boolean emptyField = true;
 
     public MessageColumnIORecordConsumer(ColumnWriteStore columns) {
       int maxDepth = 0;
@@ -76,10 +97,10 @@
     }
 
     public void printState() {
-      log(currentLevel+", "+currentIndex[currentLevel]+": "+Arrays.toString(currentColumnIO.getFieldPath())+" r:"+r[currentLevel]);
+      log(currentLevel + ", " + currentIndex[currentLevel] + ": " + Arrays.toString(currentColumnIO.getFieldPath()) + " r:" + r[currentLevel]);
       if (r[currentLevel] > currentColumnIO.getRepetitionLevel()) {
         // sanity check
-        throw new InvalidRecordException(r[currentLevel]+"(r) > "+currentColumnIO.getRepetitionLevel()+" ( schema r)");
+        throw new InvalidRecordException(r[currentLevel] + "(r) > " + currentColumnIO.getRepetitionLevel() + " ( schema r)");
       }
     }
 
@@ -110,26 +131,39 @@
     @Override
     public void startField(String field, int index) {
       try {
-        if (DEBUG) log("startField("+field+", "+index+")");
+        if (DEBUG) log("startField(" + field + ", " + index + ")");
         writeNullForMissingFields(index - 1);
         currentColumnIO = ((GroupColumnIO)currentColumnIO).getChild(index);
         currentIndex[currentLevel] = index;
+        emptyField = true;
         if (DEBUG) printState();
       } catch (RuntimeException e) {
         throw new ParquetEncodingException("error starting field " + field + " at " + index, e);
       }
     }
 
+    @Override
+    public void endField(String field, int index) {
+      if (DEBUG) log("endField(" + field + ", " + index + ")");
+      currentColumnIO = currentColumnIO.getParent();
+      if (emptyField) {
+        throw new ParquetEncodingException("empty fields are illegal, the field should be ommited completely instead");
+      }
+      currentIndex[currentLevel] = index + 1;
+      r[currentLevel] = currentLevel == 0 ? 0 : r[currentLevel - 1];
+      if (DEBUG) printState();
+    }
+
     private void writeNullForMissingFields(final int to) {
       final int from = currentIndex[currentLevel];
       for (;currentIndex[currentLevel]<=to; ++currentIndex[currentLevel]) {
         try {
           ColumnIO undefinedField = ((GroupColumnIO)currentColumnIO).getChild(currentIndex[currentLevel]);
           int d = currentColumnIO.getDefinitionLevel();
-          if (DEBUG) log(Arrays.toString(undefinedField.getFieldPath())+".writeNull("+r[currentLevel]+","+d+")");
+          if (DEBUG) log(Arrays.toString(undefinedField.getFieldPath()) + ".writeNull(" + r[currentLevel] + "," + d + ")");
           writeNull(undefinedField, r[currentLevel], d);
         } catch (RuntimeException e) {
-          throw new ParquetEncodingException("error while writing nulls from " + from + " to " + to + ". current index: "+currentIndex[currentLevel], e);
+          throw new ParquetEncodingException("error while writing nulls from " + from + " to " + to + ". current index: " + currentIndex[currentLevel], e);
         }
       }
     }
@@ -148,19 +182,7 @@
 
     private void setRepetitionLevel() {
       r[currentLevel] = currentColumnIO.getRepetitionLevel();
-      if (DEBUG) log("r: "+r[currentLevel]);
-    }
-
-    @Override
-    public void endField(String field, int index) {
-      if (DEBUG) log("endField("+field+", "+index+")");
-      currentColumnIO = currentColumnIO.getParent();
-
-      currentIndex[currentLevel] = index + 1;
-
-      r[currentLevel] = currentLevel == 0 ? 0 : r[currentLevel - 1];
-
-      if (DEBUG) printState();
+      if (DEBUG) log("r: " + r[currentLevel]);
     }
 
     @Override
@@ -177,33 +199,33 @@
     @Override
     public void endGroup() {
       if (DEBUG) log("endGroup()");
+      emptyField = false;
       int lastIndex = ((GroupColumnIO)currentColumnIO).getChildrenCount() - 1;
       writeNullForMissingFields(lastIndex);
-
       -- currentLevel;
 
       setRepetitionLevel();
       if (DEBUG) printState();
     }
 
-    @Override
-    public void addInteger(int value) {
-      if (DEBUG) log("addInt("+value+")");
-
-      getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel());
-
-      setRepetitionLevel();
-      if (DEBUG) printState();
-    }
-
     private ColumnWriter getColumnWriter() {
       return columnWriter[((PrimitiveColumnIO)currentColumnIO).getId()];
     }
 
     @Override
-    public void addLong(long value) {
-      if (DEBUG) log("addLong("+value+")");
+    public void addInteger(int value) {
+      if (DEBUG) log("addInt(" + value + ")");
+      emptyField = false;
+      getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel());
 
+      setRepetitionLevel();
+      if (DEBUG) printState();
+    }
+
+    @Override
+    public void addLong(long value) {
+      if (DEBUG) log("addLong(" + value + ")");
+      emptyField = false;
       getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel());
 
       setRepetitionLevel();
@@ -212,7 +234,8 @@
 
     @Override
     public void addBoolean(boolean value) {
-      if (DEBUG) log("addBoolean("+value+")");
+      if (DEBUG) log("addBoolean(" + value + ")");
+      emptyField = false;
       getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel());
 
       setRepetitionLevel();
@@ -221,7 +244,8 @@
 
     @Override
     public void addBinary(Binary value) {
-      if (DEBUG) log("addBinary("+value.length()+" bytes)");
+      if (DEBUG) log("addBinary(" + value.length() + " bytes)");
+      emptyField = false;
       getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel());
 
       setRepetitionLevel();
@@ -230,7 +254,8 @@
 
     @Override
     public void addFloat(float value) {
-      if (DEBUG) log("addFloat("+value+")");
+      if (DEBUG) log("addFloat(" + value + ")");
+      emptyField = false;
       getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel());
 
       setRepetitionLevel();
@@ -239,7 +264,8 @@
 
     @Override
     public void addDouble(double value) {
-      if (DEBUG) log("addDouble("+value+")");
+      if (DEBUG) log("addDouble(" + value + ")");
+      emptyField = false;
       getColumnWriter().write(value, r[currentLevel], currentColumnIO.getDefinitionLevel());
 
       setRepetitionLevel();
diff --git a/parquet-column/src/main/java/parquet/io/RecordReaderImplementation.java b/parquet-column/src/main/java/parquet/io/RecordReaderImplementation.java
index 6564d04..a4a993b 100644
--- a/parquet-column/src/main/java/parquet/io/RecordReaderImplementation.java
+++ b/parquet-column/src/main/java/parquet/io/RecordReaderImplementation.java
@@ -222,25 +222,29 @@
     public Case getCase(int currentLevel, int d, int nextR) {
       return caseLookup[currentLevel][d][nextR];
     }
+
+    public State getNextState(int nextR) {
+      return nextState[nextR];
+    }
   }
 
   private final GroupConverter recordConsumer;
   private final RecordMaterializer<T> recordMaterializer;
 
   private State[] states;
+  private ColumnReader[] columns;
 
   /**
    *
    * @param root the root of the schema
-   * @param leaves the leaves of the schema
    * @param validating
-   * @param columns2
+   * @param columnStore
    */
   public RecordReaderImplementation(MessageColumnIO root, RecordMaterializer<T> recordMaterializer, boolean validating, ColumnReadStoreImpl columnStore) {
     this.recordMaterializer = recordMaterializer;
     this.recordConsumer = recordMaterializer.getRootConverter(); // TODO: validator(wrap(recordMaterializer), validating, root.getType());
     PrimitiveColumnIO[] leaves = root.getLeaves().toArray(new PrimitiveColumnIO[root.getLeaves().size()]);
-    ColumnReader[] columns = new ColumnReader[leaves.length];
+    columns = new ColumnReader[leaves.length];
     int[][] nextReader = new int[leaves.length][];
     int[][] nextLevel = new int[leaves.length][];
     GroupConverter[][] groupConverterPaths = new GroupConverter[leaves.length][];
@@ -298,17 +302,18 @@
       states[i] = new State(i, leaves[i], columns[i], nextLevel[i], groupConverterPaths[i], primitiveConverters[i]);
 
       int[] definitionLevelToDepth = new int[states[i].primitiveColumnIO.getDefinitionLevel() + 1];
-      int depth = 0;
       // for each possible definition level, determine the depth at which to create groups
+      final ColumnIO[] path = states[i].primitiveColumnIO.getPath();
+      int depth = 0;
       for (int d = 0; d < definitionLevelToDepth.length; ++d) {
         while (depth < (states[i].fieldPath.length - 1)
-          && d > states[i].primitiveColumnIO.getPath()[depth].getDefinitionLevel()) {
+          && d >= path[depth + 1].getDefinitionLevel()
+          ) {
           ++ depth;
         }
         definitionLevelToDepth[d] = depth - 1;
       }
       states[i].definitionLevelToDepth = definitionLevelToDepth;
-
     }
     for (int i = 0; i < leaves.length; i++) {
       State state = states[i];
@@ -389,7 +394,7 @@
       // set the current value
       if (d >= currentState.maxDefinitionLevel) {
         // not null
-        currentState.primitive.addValueToPrimitiveConverter(currentState.primitiveConverter, columnReader);
+        columnReader.writeCurrentValueToConverter();
       }
       columnReader.consume();
 
@@ -443,4 +448,8 @@
     return recordConsumer;
   }
 
+  protected Iterable<ColumnReader> getColumnReaders() {
+    // Converting the array to an iterable ensures that the array cannot be altered
+    return Arrays.asList(columns);
+  }
 }
diff --git a/parquet-column/src/main/java/parquet/io/api/Binary.java b/parquet-column/src/main/java/parquet/io/api/Binary.java
index 3d532d2..365d39e 100644
--- a/parquet-column/src/main/java/parquet/io/api/Binary.java
+++ b/parquet-column/src/main/java/parquet/io/api/Binary.java
@@ -15,12 +15,16 @@
  */
 package parquet.io.api;
 
+import static parquet.bytes.BytesUtils.UTF8;
+
 import java.io.IOException;
 import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
 import parquet.bytes.BytesUtils;
+import parquet.io.ParquetEncodingException;
 
 abstract public class Binary {
 
@@ -34,7 +38,9 @@
     return new Binary() {
       @Override
       public String toStringUsingUTF8() {
-        return new String(value, offset, length, BytesUtils.UTF8);
+        return UTF8.decode(ByteBuffer.wrap(value, offset, length)).toString();
+        // TODO: figure out why the following line was much slower
+        // return new String(value, offset, length, BytesUtils.UTF8);
       }
 
       @Override
@@ -119,8 +125,75 @@
     };
   }
 
+  public static Binary fromByteBuffer(final ByteBuffer value) {
+    return new Binary() {
+      @Override
+      public String toStringUsingUTF8() {
+        return new String(getBytes(), BytesUtils.UTF8);
+      }
+
+      @Override
+      public int length() {
+        return value.remaining();
+      }
+
+      @Override
+      public void writeTo(OutputStream out) throws IOException {
+        out.write(getBytes());
+      }
+
+      @Override
+      public byte[] getBytes() {
+        byte[] bytes = new byte[value.remaining()];
+
+        value.mark();
+        value.get(bytes).reset();
+        return bytes;
+      }
+
+      @Override
+      public int hashCode() {
+        if (value.hasArray()) {
+          return Binary.hashCode(value.array(), value.arrayOffset() + value.position(),
+              value.arrayOffset() + value.remaining());
+        }
+        byte[] bytes = getBytes();
+        return Binary.hashCode(bytes, 0, bytes.length);
+      }
+
+      @Override
+      boolean equals(Binary other) {
+        if (value.hasArray()) {
+          return other.equals(value.array(), value.arrayOffset() + value.position(),
+              value.arrayOffset() + value.remaining());
+        }
+        byte[] bytes = getBytes();
+        return other.equals(bytes, 0, bytes.length);
+      }
+
+      @Override
+      boolean equals(byte[] other, int otherOffset, int otherLength) {
+        if (value.hasArray()) {
+          return Binary.equals(value.array(), value.arrayOffset() + value.position(),
+              value.arrayOffset() + value.remaining(), other, otherOffset, otherLength);
+        }
+        byte[] bytes = getBytes();
+        return Binary.equals(bytes, 0, bytes.length, other, otherOffset, otherLength);
+      }
+
+      @Override
+      public ByteBuffer toByteBuffer() {
+        return value;
+      }
+    };
+  }
+
   public static Binary fromString(final String value) {
-    return fromByteArray(value.getBytes(BytesUtils.UTF8));
+    try {
+      return fromByteArray(value.getBytes("UTF-8"));
+    } catch (UnsupportedEncodingException e) {
+      throw new ParquetEncodingException("UTF-8 not supported.", e);
+    }
   }
 
   /**
@@ -186,4 +259,7 @@
 
   abstract public ByteBuffer toByteBuffer();
 
+  public String toString() {
+    return "Binary{" + length() + " bytes, " + toStringUsingUTF8() + "}";
+  };
 }
diff --git a/parquet-column/src/main/java/parquet/io/api/GroupConverter.java b/parquet-column/src/main/java/parquet/io/api/GroupConverter.java
index 5c37a42..6561e59 100644
--- a/parquet-column/src/main/java/parquet/io/api/GroupConverter.java
+++ b/parquet-column/src/main/java/parquet/io/api/GroupConverter.java
@@ -15,6 +15,7 @@
  */
 package parquet.io.api;
 
+
 /**
  * converter for group nodes
  *
@@ -36,7 +37,7 @@
   /**
    * called at initialization based on schema
    * must consistently return the same object
-   * @param fieldIndex index of a group field in this group
+   * @param fieldIndex index of the field in this group
    * @return the corresponding converter
    */
   abstract public Converter getConverter(int fieldIndex);
diff --git a/parquet-column/src/main/java/parquet/io/api/PrimitiveConverter.java b/parquet-column/src/main/java/parquet/io/api/PrimitiveConverter.java
index 26326d4..4c4b480 100644
--- a/parquet-column/src/main/java/parquet/io/api/PrimitiveConverter.java
+++ b/parquet-column/src/main/java/parquet/io/api/PrimitiveConverter.java
@@ -15,6 +15,7 @@
  */
 package parquet.io.api;
 
+import parquet.column.Dictionary;
 
 /**
  * converter for leaves of the schema
@@ -34,10 +35,35 @@
     return this;
   }
 
+  /**
+   * if it returns true we will attempt to use dictionary based conversion instead
+   * @return if dictionary is supported
+   */
+  public boolean hasDictionarySupport() {
+    return false;
+  }
+
+  /**
+   * Set the dictionary to use if the data was encoded using dictionary encoding
+   * and the converter hasDictionarySupport().
+   * @param dictionary the dictionary to use for conversion
+   */
+  public void setDictionary(Dictionary dictionary) {
+    throw new UnsupportedOperationException(getClass().getName());
+  }
+
   /** runtime calls  **/
 
   /**
-   * @param fieldIndex index of the field
+   * add a value based on the dictionary set with setDictionary()
+   * Will be used if the Converter has dictionary support and the data was encoded using a dictionary
+   * @param dictionaryId the id in the dictionary of the value to add
+   */
+  public void addValueFromDictionary(int dictionaryId) {
+    throw new UnsupportedOperationException(getClass().getName());
+  }
+
+  /**
    * @param value value to set
    */
   public void addBinary(Binary value) {
@@ -45,7 +71,6 @@
   }
 
   /**
-   * @param fieldIndex index of the field
    * @param value value to set
    */
   public void addBoolean(boolean value) {
@@ -53,7 +78,6 @@
   }
 
   /**
-   * @param fieldIndex index of the field
    * @param value value to set
    */
   public void addDouble(double value) {
@@ -61,7 +85,6 @@
   }
 
   /**
-   * @param fieldIndex index of the field
    * @param value value to set
    */
   public void addFloat(float value) {
@@ -69,7 +92,6 @@
   }
 
   /**
-   * @param fieldIndex index of the field
    * @param value value to set
    */
   public void addInt(int value) {
@@ -77,7 +99,6 @@
   }
 
   /**
-   * @param fieldIndex index of the field
    * @param value value to set
    */
   public void addLong(long value) {
diff --git a/parquet-column/src/main/java/parquet/schema/GroupType.java b/parquet-column/src/main/java/parquet/schema/GroupType.java
index 6af2e30..06d99da 100644
--- a/parquet-column/src/main/java/parquet/schema/GroupType.java
+++ b/parquet-column/src/main/java/parquet/schema/GroupType.java
@@ -222,6 +222,14 @@
   }
 
   @Override
+  protected boolean containsPath(String[] path, int depth) {
+    if (depth == path.length) {
+      return false;
+    }
+    return containsField(path[depth]) && getType(path[depth]).containsPath(path, depth + 1);
+  }
+
+  @Override
   protected List<String[]> getPaths(int depth) {
     List<String[]> result = new ArrayList<String[]>();
     for (Type field : fields) {
diff --git a/parquet-column/src/main/java/parquet/schema/MessageType.java b/parquet-column/src/main/java/parquet/schema/MessageType.java
index 316e9de..3f0deb1 100644
--- a/parquet-column/src/main/java/parquet/schema/MessageType.java
+++ b/parquet-column/src/main/java/parquet/schema/MessageType.java
@@ -124,4 +124,8 @@
     path.add(this);
     return converter.convertMessageType(this, convertChildren(path, converter));
   }
+
+  public boolean containsPath(String[] path) {
+    return containsPath(path, 0);
+  }
 }
diff --git a/parquet-column/src/main/java/parquet/schema/PrimitiveType.java b/parquet-column/src/main/java/parquet/schema/PrimitiveType.java
index c0bf1ac..bf56df7 100644
--- a/parquet-column/src/main/java/parquet/schema/PrimitiveType.java
+++ b/parquet-column/src/main/java/parquet/schema/PrimitiveType.java
@@ -34,13 +34,34 @@
  */
 public final class PrimitiveType extends Type {
 
+  public static interface PrimitiveTypeNameConverter<T> {
+
+    T convertFLOAT(PrimitiveTypeName primitiveTypeName);
+
+    T convertDOUBLE(PrimitiveTypeName primitiveTypeName);
+
+    T convertINT32(PrimitiveTypeName primitiveTypeName);
+
+    T convertINT64(PrimitiveTypeName primitiveTypeName);
+
+    T convertINT96(PrimitiveTypeName primitiveTypeName);
+
+    T convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName);
+
+    T convertBOOLEAN(PrimitiveTypeName primitiveTypeName);
+
+    T convertBINARY(PrimitiveTypeName primitiveTypeName);
+
+  }
+
   /**
    * Supported Primitive types
    *
    * @author Julien Le Dem
    */
   public static enum PrimitiveTypeName {
-      INT64("Long", Long.TYPE) {
+
+    INT64("Long", Long.TYPE) {
       @Override
       public String toString(ColumnReader columnReader) {
         return String.valueOf(columnReader.getLong());
@@ -57,6 +78,11 @@
           PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
         primitiveConverter.addLong(columnReader.getLong());
       }
+
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertINT64(this);
+      }
     },
     INT32("Integer", Integer.TYPE) {
       @Override
@@ -75,6 +101,11 @@
           PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
         primitiveConverter.addInt(columnReader.getInteger());
       }
+
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertINT32(this);
+      }
     },
     BOOLEAN("Boolean", Boolean.TYPE) {
       @Override
@@ -93,6 +124,11 @@
           PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
         primitiveConverter.addBoolean(columnReader.getBoolean());
       }
+
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertBOOLEAN(this);
+      }
     },
     BINARY("Binary", Binary.class) {
       @Override
@@ -111,6 +147,11 @@
           PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
         primitiveConverter.addBinary(columnReader.getBinary());
       }
+
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertBINARY(this);
+      }
     },
     FLOAT("Float", Float.TYPE) {
       @Override
@@ -129,6 +170,11 @@
           PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
         primitiveConverter.addFloat(columnReader.getFloat());
       }
+
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertFLOAT(this);
+      }
     },
     DOUBLE("Double", Double.TYPE) {
       @Override
@@ -147,6 +193,11 @@
           PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
         primitiveConverter.addDouble(columnReader.getDouble());
       }
+
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertDOUBLE(this);
+      }
     },
     INT96(null, null) { // TODO: support for INT96
       @Override
@@ -163,6 +214,11 @@
           PrimitiveConverter primitiveConverter, ColumnReader columnReader) {
         throw new UnsupportedOperationException("NYI");
       }
+
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertINT96(this);
+      }
     },
     FIXED_LEN_BYTE_ARRAY(null, null) { // TODO: support for FIXED_LEN_BYTE_ARRAY
 
@@ -183,7 +239,11 @@
         throw new UnsupportedOperationException("NYI");
       }
 
-    } ;
+      @Override
+      public <T> T convert(PrimitiveTypeNameConverter<T> converter) {
+        return converter.convertFIXED_LEN_BYTE_ARRAY(this);
+      }
+    };
 
     private final String internalName;
     public final Class<?> javaType;
@@ -220,6 +280,8 @@
     abstract public void addValueToPrimitiveConverter(
         PrimitiveConverter primitiveConverter, ColumnReader columnReader);
 
+    abstract public <T> T convert(PrimitiveTypeNameConverter<T> converter);
+
   }
 
   private final PrimitiveTypeName primitive;
@@ -268,11 +330,11 @@
   @Override
   public void writeToStringBuilder(StringBuilder sb, String indent) {
     sb.append(indent)
-        .append(getRepetition().name().toLowerCase())
-        .append(" ")
-        .append(primitive.name().toLowerCase())
-        .append(" ")
-        .append(getName());
+    .append(getRepetition().name().toLowerCase())
+    .append(" ")
+    .append(primitive.name().toLowerCase())
+    .append(" ")
+    .append(getName());
     if (getOriginalType() != null) {
       sb.append(" (").append(getOriginalType()).append(")");
     }
@@ -351,4 +413,9 @@
   public <T> T convert(List<GroupType> path, TypeConverter<T> converter) {
     return converter.convertPrimitiveType(path, this);
   }
+
+  @Override
+  protected boolean containsPath(String[] path, int depth) {
+    return path.length == depth;
+  }
 }
diff --git a/parquet-column/src/main/java/parquet/schema/Type.java b/parquet-column/src/main/java/parquet/schema/Type.java
index 7980b57..1b4d851 100644
--- a/parquet-column/src/main/java/parquet/schema/Type.java
+++ b/parquet-column/src/main/java/parquet/schema/Type.java
@@ -28,9 +28,27 @@
 abstract public class Type {
 
   public static enum Repetition {
-    REQUIRED, // exactly 1
-    OPTIONAL, // 0 or 1
-    REPEATED  // 0 or more
+    REQUIRED { // exactly 1
+      @Override
+      public boolean isMoreRestrictiveThan(Repetition other) {
+        return other != REQUIRED;
+      }
+    },
+    OPTIONAL { // 0 or 1
+      @Override
+      public boolean isMoreRestrictiveThan(Repetition other) {
+        return other == REPEATED;
+      }
+    },
+    REPEATED {  // 0 or more
+      @Override
+      public boolean isMoreRestrictiveThan(Repetition other) {
+        return false;
+      }
+    }
+    ;
+
+    abstract public boolean isMoreRestrictiveThan(Repetition other);
   }
 
   private final String name;
@@ -114,6 +132,8 @@
 
   protected abstract List<String[]> getPaths(int depth);
 
+  protected abstract boolean containsPath(String[] path, int depth);
+
   /**
    * {@inheritDoc}
    */
diff --git a/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java b/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java
index d621457..cb8e41b 100644
--- a/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java
+++ b/parquet-column/src/test/java/parquet/column/mem/TestMemColumn.java
@@ -26,6 +26,7 @@
 import parquet.column.impl.ColumnReadStoreImpl;
 import parquet.column.impl.ColumnWriteStoreImpl;
 import parquet.column.page.mem.MemPageStore;
+import parquet.example.DummyRecordConverter;
 import parquet.io.api.Binary;
 import parquet.schema.MessageType;
 import parquet.schema.MessageTypeParser;
@@ -36,16 +37,14 @@
 
   @Test
   public void testMemColumn() throws Exception {
-    String schema = "message msg { required group foo { required int64 bar; } }";
-    String[] col = {"foo", "bar"};
+    MessageType schema = MessageTypeParser.parseMessageType("message msg { required group foo { required int64 bar; } }");
+    ColumnDescriptor path = schema.getColumnDescription(new String[] {"foo", "bar"});
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048);
-    ColumnDescriptor path = getCol(schema, col);
-    ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
+    ColumnWriter columnWriter = getColumnWriter(path, memPageStore);
     columnWriter.write(42l, 0, 0);
     columnWriter.flush();
 
-    ColumnReader columnReader = new ColumnReadStoreImpl(memPageStore).getColumnReader(path);
+    ColumnReader columnReader = getColumnReader(memPageStore, path, schema);
     while (!columnReader.isFullyConsumed()) {
       assertEquals(columnReader.getCurrentRepetitionLevel(), 0);
       assertEquals(columnReader.getCurrentDefinitionLevel(), 0);
@@ -54,25 +53,35 @@
     }
   }
 
-  private ColumnDescriptor getCol(String schema, String[] col) {
-    MessageType mt = MessageTypeParser.parseMessageType(schema);
-    ColumnDescriptor path = mt.getColumnDescription(col);
-    return path;
+  private ColumnWriter getColumnWriter(ColumnDescriptor path, MemPageStore memPageStore) {
+    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+    ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
+    return columnWriter;
+  }
+
+  private ColumnReader getColumnReader(MemPageStore memPageStore, ColumnDescriptor path, MessageType schema) {
+    return new ColumnReadStoreImpl(
+        memPageStore,
+        new DummyRecordConverter(schema).getRootConverter(),
+        schema
+        ).getColumnReader(path);
   }
 
   @Test
   public void testMemColumnBinary() throws Exception {
-    String schema = "message msg { required group foo { required binary bar; } }";
+    MessageType mt = MessageTypeParser.parseMessageType("message msg { required group foo { required binary bar; } }");
     String[] col = new String[]{"foo", "bar"};
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048);
-    ColumnDescriptor path = getCol(schema, col);
+
+    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+    ColumnDescriptor path1 = mt.getColumnDescription(col);
+    ColumnDescriptor path = path1;
 
     ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
     columnWriter.write(Binary.fromString("42"), 0, 0);
     columnWriter.flush();
 
-    ColumnReader columnReader = new ColumnReadStoreImpl(memPageStore).getColumnReader(path);
+    ColumnReader columnReader = getColumnReader(memPageStore, path, mt);
     while (!columnReader.isFullyConsumed()) {
       assertEquals(columnReader.getCurrentRepetitionLevel(), 0);
       assertEquals(columnReader.getCurrentDefinitionLevel(), 0);
@@ -83,11 +92,12 @@
 
   @Test
   public void testMemColumnSeveralPages() throws Exception {
-    String schema = "message msg { required group foo { required int64 bar; } }";
+    MessageType mt = MessageTypeParser.parseMessageType("message msg { required group foo { required int64 bar; } }");
     String[] col = new String[]{"foo", "bar"};
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048);
-    ColumnDescriptor path = getCol(schema, col);
+    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+    ColumnDescriptor path1 = mt.getColumnDescription(col);
+    ColumnDescriptor path = path1;
 
     ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
     for (int i = 0; i < 2000; i++) {
@@ -95,7 +105,7 @@
     }
     columnWriter.flush();
 
-    ColumnReader columnReader = new ColumnReadStoreImpl(memPageStore).getColumnReader(path);
+    ColumnReader columnReader = getColumnReader(memPageStore, path, mt);
     while (!columnReader.isFullyConsumed()) {
       assertEquals(columnReader.getCurrentRepetitionLevel(), 0);
       assertEquals(columnReader.getCurrentDefinitionLevel(), 0);
@@ -106,11 +116,12 @@
 
   @Test
   public void testMemColumnSeveralPagesRepeated() throws Exception {
-    String schema = "message msg { repeated group foo { repeated int64 bar; } }";
+    MessageType mt = MessageTypeParser.parseMessageType("message msg { repeated group foo { repeated int64 bar; } }");
     String[] col = new String[]{"foo", "bar"};
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048);
-    ColumnDescriptor path = getCol(schema, col);
+    ColumnWriteStoreImpl memColumnsStore = new ColumnWriteStoreImpl(memPageStore, 2048, 2048, false);
+    ColumnDescriptor path1 = mt.getColumnDescription(col);
+    ColumnDescriptor path = path1;
 
     ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
     int[] rs = { 0, 0, 0, 1, 1, 1, 2, 2, 2};
@@ -127,7 +138,7 @@
     }
     columnWriter.flush();
 
-    ColumnReader columnReader = new ColumnReadStoreImpl(memPageStore).getColumnReader(path);
+    ColumnReader columnReader = getColumnReader(memPageStore, path, mt);
     int i = 0;
     while (!columnReader.isFullyConsumed()) {
       int r = rs[i % rs.length];
diff --git a/parquet-column/src/test/java/parquet/column/page/mem/MemPageReader.java b/parquet-column/src/test/java/parquet/column/page/mem/MemPageReader.java
index 3689951..e6a5d7a 100644
--- a/parquet-column/src/test/java/parquet/column/page/mem/MemPageReader.java
+++ b/parquet-column/src/test/java/parquet/column/page/mem/MemPageReader.java
@@ -16,10 +16,12 @@
 package parquet.column.page.mem;
 
 import static parquet.Log.DEBUG;
+import static parquet.Preconditions.checkNotNull;
 
 import java.util.Iterator;
 
 import parquet.Log;
+import parquet.column.page.DictionaryPage;
 import parquet.column.page.Page;
 import parquet.column.page.PageReader;
 import parquet.io.ParquetDecodingException;
@@ -28,16 +30,16 @@
 public class MemPageReader implements PageReader {
   private static final Log LOG = Log.getLog(MemPageReader.class);
 
-  private long totalValueCount;
-  private Iterator<Page> pages;
+  private final long totalValueCount;
+  private final Iterator<Page> pages;
+  private final DictionaryPage dictionaryPage;
 
-  public MemPageReader(long totalValueCount, Iterator<Page> pages) {
+  public MemPageReader(long totalValueCount, Iterator<Page> pages, DictionaryPage dictionaryPage) {
     super();
-    if (pages == null) {
-      throw new NullPointerException("pages");
-    }
+    checkNotNull(pages, "pages");
     this.totalValueCount = totalValueCount;
     this.pages = pages;
+    this.dictionaryPage = dictionaryPage;
   }
 
   @Override
@@ -56,4 +58,9 @@
     }
   }
 
+  @Override
+  public DictionaryPage readDictionaryPage() {
+    return dictionaryPage;
+  }
+
 }
diff --git a/parquet-column/src/test/java/parquet/column/page/mem/MemPageStore.java b/parquet-column/src/test/java/parquet/column/page/mem/MemPageStore.java
index 71ce1f5..cdf02f7 100644
--- a/parquet-column/src/test/java/parquet/column/page/mem/MemPageStore.java
+++ b/parquet-column/src/test/java/parquet/column/page/mem/MemPageStore.java
@@ -53,7 +53,7 @@
     }
     List<Page> pages = new ArrayList<Page>(pageWriter.getPages());
     if (Log.DEBUG) LOG.debug("initialize page reader with "+ pageWriter.getTotalValueCount() + " values and " + pages.size() + " pages");
-    return new MemPageReader(pageWriter.getTotalValueCount(), pages.iterator());
+    return new MemPageReader(pageWriter.getTotalValueCount(), pages.iterator(), pageWriter.getDictionaryPage());
   }
 
   @Override
diff --git a/parquet-column/src/test/java/parquet/column/page/mem/MemPageWriter.java b/parquet-column/src/test/java/parquet/column/page/mem/MemPageWriter.java
index 0de399d..164cde0 100644
--- a/parquet-column/src/test/java/parquet/column/page/mem/MemPageWriter.java
+++ b/parquet-column/src/test/java/parquet/column/page/mem/MemPageWriter.java
@@ -24,6 +24,7 @@
 import parquet.Log;
 import parquet.bytes.BytesInput;
 import parquet.column.Encoding;
+import parquet.column.page.DictionaryPage;
 import parquet.column.page.Page;
 import parquet.column.page.PageWriter;
 import parquet.io.ParquetEncodingException;
@@ -33,6 +34,7 @@
   private static final Log LOG = Log.getLog(MemPageWriter.class);
 
   private final List<Page> pages = new ArrayList<Page>();
+  private DictionaryPage dictionaryPage;
   private long memSize = 0;
   private long totalValueCount = 0;
 
@@ -58,6 +60,10 @@
     return pages;
   }
 
+  public DictionaryPage getDictionaryPage() {
+    return dictionaryPage;
+  }
+
   public long getTotalValueCount() {
     return totalValueCount;
   }
@@ -68,4 +74,20 @@
     return memSize;
   }
 
+  @Override
+  public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException {
+    if (this.dictionaryPage != null) {
+      throw new ParquetEncodingException("Only one dictionary page per block");
+    }
+    this.memSize += dictionaryPage.getBytes().size();
+    this.dictionaryPage = dictionaryPage.copy();
+    if (DEBUG) LOG.debug("dictionary page written for " + dictionaryPage.getBytes().size() + " bytes and " + dictionaryPage.getDictionarySize() + " records");
+  }
+
+  @Override
+  public String memUsageString(String prefix) {
+    return String.format("%s %,d bytes", prefix, memSize);
+
+  }
+
 }
diff --git a/parquet-column/src/test/java/parquet/column/values/bitpacking/BitPackingPerfTest.java b/parquet-column/src/test/java/parquet/column/values/bitpacking/BitPackingPerfTest.java
new file mode 100644
index 0000000..0a3ccc1
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/bitpacking/BitPackingPerfTest.java
@@ -0,0 +1,100 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import parquet.column.values.ValuesReader;
+import parquet.column.values.bitpacking.BitPacking.BitPackingWriter;
+
+/**
+ * Improvable micro benchmark for bitpacking
+ * run with: -verbose:gc -Xmx2g -Xms2g
+ * @author Julien Le Dem
+ *
+ */
+public class BitPackingPerfTest {
+
+  public static void main(String[] args) throws IOException {
+    int COUNT = 800000;
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    BitPackingWriter w = BitPacking.getBitPackingWriter(1, baos);
+    long t0 = System.currentTimeMillis();
+    for (int i = 0 ; i < COUNT; ++i) {
+      w.write(i % 2);
+    }
+    w.finish();
+    long t1 = System.currentTimeMillis();
+    System.out.println("written in " + (t1 - t0) + "ms");
+    System.out.println();
+    byte[] bytes = baos.toByteArray();
+    System.out.println(bytes.length);
+    int[] result = new int[COUNT];
+    for (int l = 0; l < 5; l++) {
+      long s = manual(bytes, result);
+      long b = generated(bytes, result);
+      float ratio = (float)b/s;
+      System.out.println("                                             " + ratio + (ratio < 1 ? " < 1 => GOOD" : " >= 1 => BAD"));
+    }
+  }
+
+  private static void verify(int[] result) {
+    int error = 0;
+    for (int i = 0 ; i < result.length; ++i) {
+      if (result[i] != i % 2) {
+        error ++;
+      }
+    }
+    if (error != 0) {
+      throw new RuntimeException("errors: " + error + " / " + result.length);
+    }
+  }
+
+  private static long manual(byte[] bytes, int[] result)
+      throws IOException {
+    return readNTimes(bytes, result, new BitPackingValuesReader(1));
+  }
+
+  private static long generated(byte[] bytes, int[] result)
+      throws IOException {
+    return readNTimes(bytes, result, new ByteBitPackingValuesReader(1, Packer.BIG_ENDIAN));
+  }
+
+  private static long readNTimes(byte[] bytes, int[] result, ValuesReader r)
+      throws IOException {
+    System.out.println();
+    long t = 0;
+    int N = 10;
+    System.gc();
+    System.out.print("                                             " + r.getClass().getSimpleName());
+    System.out.print(" no gc <");
+    for (int k = 0; k < N; k++) {
+      long t2 = System.nanoTime();
+      r.initFromPage(result.length, bytes, 0);
+      for (int i = 0; i < result.length; i++) {
+        result[i] = r.readInteger();
+      }
+      long t3 = System.nanoTime();
+      t += t3 - t2;
+    }
+    System.out.println("> read in " + t/1000 + "µs " + (N * result.length / (t / 1000)) + " values per µs");
+    verify(result);
+    return t;
+  }
+
+}
+
diff --git a/parquet-column/src/test/java/parquet/column/primitive/TestBitPackingColumn.java b/parquet-column/src/test/java/parquet/column/values/bitpacking/TestBitPackingColumn.java
similarity index 71%
rename from parquet-column/src/test/java/parquet/column/primitive/TestBitPackingColumn.java
rename to parquet-column/src/test/java/parquet/column/values/bitpacking/TestBitPackingColumn.java
index 09e394e..2b947eb 100644
--- a/parquet-column/src/test/java/parquet/column/primitive/TestBitPackingColumn.java
+++ b/parquet-column/src/test/java/parquet/column/values/bitpacking/TestBitPackingColumn.java
@@ -13,17 +13,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.primitive;
+package parquet.column.values.bitpacking;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static parquet.column.values.bitpacking.Packer.BIG_ENDIAN;
 
 import java.io.IOException;
 
 import org.junit.Test;
 
-import parquet.column.values.bitpacking.BitPackingValuesReader;
-import parquet.column.values.bitpacking.BitPackingValuesWriter;
+import parquet.column.values.ValuesReader;
+import parquet.column.values.ValuesWriter;
 
 public class TestBitPackingColumn {
 
@@ -154,23 +155,49 @@
   }
 
   private void validateEncodeDecode(int bitLength, int[] vals, String expected) throws IOException {
-    final int bound = (int)Math.pow(2, bitLength) - 1;
-    BitPackingValuesWriter w = new BitPackingValuesWriter(bound);
-    for (int i : vals) {
-      w.writeInteger(i);
+    for (PACKING_TYPE type : PACKING_TYPE.values()) {
+      System.out.println(type);
+      final int bound = (int)Math.pow(2, bitLength) - 1;
+      ValuesWriter w = type.getWriter(bound);
+      for (int i : vals) {
+        w.writeInteger(i);
+      }
+      byte[] bytes = w.getBytes().toByteArray();
+      System.out.println("vals ("+bitLength+"): " + TestBitPacking.toString(vals));
+      System.out.println("bytes: " + TestBitPacking.toString(bytes));
+      assertEquals(type.toString(), expected, TestBitPacking.toString(bytes));
+      ValuesReader r = type.getReader(bound);
+      r.initFromPage(vals.length, bytes, 0);
+      int[] result = new int[vals.length];
+      for (int i = 0; i < result.length; i++) {
+        result[i] = r.readInteger();
+      }
+      System.out.println("result: " + TestBitPacking.toString(result));
+      assertArrayEquals(type + " result: " + TestBitPacking.toString(result), vals, result);
     }
-    byte[] bytes = w.getBytes().toByteArray();
-    System.out.println("vals ("+bitLength+"): " + TestBitPacking.toString(vals));
-    System.out.println("bytes: " + TestBitPacking.toString(bytes));
-    assertEquals(expected, TestBitPacking.toString(bytes));
-    BitPackingValuesReader r = new BitPackingValuesReader(bound);
-    r.initFromPage(vals.length, bytes, 0);
-    int[] result = new int[vals.length];
-    for (int i = 0; i < result.length; i++) {
-      result[i] = r.readInteger();
+  }
+
+  private static enum PACKING_TYPE {
+    BYTE_BASED_MANUAL {
+      public ValuesReader getReader(final int bound) {
+        return new BitPackingValuesReader(bound);
+      }
+      public ValuesWriter getWriter(final int bound) {
+        return new BitPackingValuesWriter(bound, 32*1024);
+      }
     }
-    System.out.println("result: " + TestBitPacking.toString(result));
-    assertArrayEquals("result: " + TestBitPacking.toString(result), vals, result);
+    ,
+    BYTE_BASED_GENERATED {
+      public ValuesReader getReader(final int bound) {
+        return new ByteBitPackingValuesReader(bound, BIG_ENDIAN);
+      }
+      public ValuesWriter getWriter(final int bound) {
+        return new ByteBitPackingValuesWriter(bound, BIG_ENDIAN);
+      }
+    }
+    ;
+    abstract public ValuesReader getReader(final int bound);
+    abstract public ValuesWriter getWriter(final int bound);
   }
 
 }
diff --git a/parquet-column/src/test/java/parquet/column/values/boundedint/TestBoundedColumns.java b/parquet-column/src/test/java/parquet/column/values/boundedint/TestBoundedColumns.java
index 93a3a64..9c6e054 100644
--- a/parquet-column/src/test/java/parquet/column/values/boundedint/TestBoundedColumns.java
+++ b/parquet-column/src/test/java/parquet/column/values/boundedint/TestBoundedColumns.java
@@ -54,7 +54,7 @@
   }
 
   private void compareOutput(int bound, int[] ints, String[] result) throws IOException {
-    BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound);
+    BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound, 64*1024);
     for (int i : ints) {
       bicw.writeInteger(i);
     }
@@ -123,7 +123,7 @@
       ByteArrayOutputStream tmp = new ByteArrayOutputStream();
 
       int[] stream = new int[totalValuesInStream];
-      BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound);
+      BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound, 64 * 1024);
       int idx = 0;
       for (int stripeNum = 0; stripeNum < valuesPerStripe.length; stripeNum++) {
         int next = 0;
diff --git a/parquet-column/src/test/java/parquet/column/values/dictionary/TestDictionary.java b/parquet-column/src/test/java/parquet/column/values/dictionary/TestDictionary.java
new file mode 100644
index 0000000..f1357fe
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/dictionary/TestDictionary.java
@@ -0,0 +1,131 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.dictionary;
+
+import static org.junit.Assert.assertEquals;
+import static parquet.column.Encoding.PLAIN_DICTIONARY;
+
+import java.io.IOException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import parquet.bytes.BytesInput;
+import parquet.column.ColumnDescriptor;
+import parquet.column.Dictionary;
+import parquet.column.Encoding;
+import parquet.column.page.DictionaryPage;
+import parquet.column.values.ValuesReader;
+import parquet.column.values.plain.BinaryPlainValuesReader;
+import parquet.io.api.Binary;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+
+public class TestDictionary {
+
+  @Test
+  public void testDict() throws IOException {
+    int COUNT = 100;
+    DictionaryValuesWriter cw = new DictionaryValuesWriter(10000, 10000);
+    for (int i = 0; i < COUNT; i++) {
+      cw.writeBytes(Binary.fromString("a" + i % 10));
+    }
+    assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+    final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
+    cw.reset();
+    for (int i = 0; i < COUNT; i++) {
+      cw.writeBytes(Binary.fromString("b" + i % 10));
+    }
+    assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+    final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
+    cw.reset();
+
+    final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
+    final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"foo"}, PrimitiveTypeName.BINARY, 0, 0);
+    final Dictionary dictionary = PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
+//    System.out.println(dictionary);
+    final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
+
+    cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
+    for (int i = 0; i < COUNT; i++) {
+      final String str = cr.readBytes().toStringUsingUTF8();
+      Assert.assertEquals("a" + i % 10, str);
+    }
+
+    cr.initFromPage(COUNT, bytes2.toByteArray(), 0);
+    for (int i = 0; i < COUNT; i++) {
+      final String str = cr.readBytes().toStringUsingUTF8();
+      Assert.assertEquals("b" + i % 10, str);
+    }
+
+  }
+
+  @Test
+  public void testDictInefficiency() throws IOException {
+    int COUNT = 40000;
+    DictionaryValuesWriter cw = new DictionaryValuesWriter(200000000, 1100000);
+    for (int i = 0; i < COUNT; i++) {
+      cw.writeBytes(Binary.fromString("a" + i ));
+    }
+    final BytesInput bytes1 = BytesInput.copy(cw.getBytes());
+    final Encoding encoding1 = cw.getEncoding();
+    System.out.println(encoding1 + "  " + bytes1.size());
+    cw.reset();
+    for (int i = 0; i < COUNT; i++) {
+      cw.writeBytes(Binary.fromString("b" + i ));
+    }
+    final BytesInput bytes2 = BytesInput.copy(cw.getBytes());
+    final Encoding encoding2 = cw.getEncoding();
+    System.out.println(encoding2 + "  " + bytes2.size());
+    cw.reset();
+
+    final DictionaryPage dictionaryPage = cw.createDictionaryPage();
+    Dictionary dictionary = null;
+    ValuesReader cr;
+    if (dictionaryPage != null) {
+      System.out.println("dict byte size: " + dictionaryPage.getBytes().size());
+      final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"foo"}, PrimitiveTypeName.BINARY, 0, 0);
+      dictionary = Encoding.PLAIN_DICTIONARY.initDictionary(descriptor, dictionaryPage);
+
+      cr = new DictionaryValuesReader(dictionary);
+    } else {
+      cr = new BinaryPlainValuesReader();
+    }
+
+    if (dictionary != null && encoding1 == Encoding.PLAIN_DICTIONARY) {
+      cr = new DictionaryValuesReader(dictionary);
+    } else {
+      cr = new BinaryPlainValuesReader();
+    }
+
+    cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
+    for (int i = 0; i < COUNT; i++) {
+      final String str = cr.readBytes().toStringUsingUTF8();
+      Assert.assertEquals("a" + i, str);
+    }
+
+    if (dictionary != null && encoding2 == Encoding.PLAIN_DICTIONARY) {
+      cr = new DictionaryValuesReader(dictionary);
+    } else {
+      cr = new BinaryPlainValuesReader();
+    }
+    cr.initFromPage(COUNT, bytes2.toByteArray(), 0);
+    for (int i = 0; i < COUNT; i++) {
+      final String str = cr.readBytes().toStringUsingUTF8();
+      Assert.assertEquals("b" + i, str);
+    }
+
+  }
+}
diff --git a/parquet-column/src/test/java/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java b/parquet-column/src/test/java/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
new file mode 100644
index 0000000..2058dba
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
@@ -0,0 +1,90 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.rle;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * @author Alex Levenson
+ */
+public class RunLengthBitPackingHybridIntegrationTest {
+
+  @Test
+  public void integrationTest() throws Exception {
+    for (int i = 0; i <= 32; i++) {
+      doIntegrationTest(i);
+    }
+  }
+
+  private void doIntegrationTest(int bitWidth) throws Exception {
+    long modValue = 1L << bitWidth;
+
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(bitWidth, 1000);
+
+    for (int i = 0; i < 100; i++) {
+      encoder.writeInt((int) (i % modValue));
+    }
+
+    for (int i = 0; i < 100; i++) {
+      encoder.writeInt((int) (77 % modValue));
+    }
+
+    for (int i = 0; i < 100; i++) {
+      encoder.writeInt((int) (88 % modValue));
+    }
+
+    for (int i = 0; i < 1000; i++) {
+      encoder.writeInt((int) (i % modValue));
+      encoder.writeInt((int) (i % modValue));
+      encoder.writeInt((int) (i % modValue));
+    }
+
+    for (int i = 0; i < 1000; i++) {
+      encoder.writeInt((int) (17 % modValue));
+    }
+
+    InputStream in = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    RunLengthBitPackingHybridDecoder decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);
+
+    for (int i = 0; i < 100; i++) {
+      assertEquals(i % modValue, decoder.readInt());
+    }
+
+    for (int i = 0; i < 100; i++) {
+      assertEquals(77 % modValue, decoder.readInt());
+    }
+
+    for (int i = 0; i < 100; i++) {
+      assertEquals(88 % modValue, decoder.readInt());
+    }
+
+    for (int i = 0; i < 1000; i++) {
+      assertEquals(i % modValue, decoder.readInt());
+      assertEquals(i % modValue, decoder.readInt());
+      assertEquals(i % modValue, decoder.readInt());
+    }
+
+    for (int i = 0; i < 1000; i++) {
+      assertEquals(17 % modValue, decoder.readInt());
+    }
+  }
+}
diff --git a/parquet-column/src/test/java/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java b/parquet-column/src/test/java/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
new file mode 100644
index 0000000..21ebeae
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
@@ -0,0 +1,303 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.rle;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayInputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Test;
+
+import parquet.bytes.BytesUtils;
+import parquet.column.values.bitpacking.BytePacker;
+import parquet.column.values.bitpacking.Packer;
+
+/**
+ * @author Alex Levenson
+ */
+public class TestRunLengthBitPackingHybridEncoder {
+
+  @Test
+  public void testRLEOnly() throws Exception {
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5);
+    for (int i = 0; i < 100; i++) {
+      encoder.writeInt(4);
+    }
+    for (int i = 0; i < 100; i++) {
+      encoder.writeInt(5);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // header = 100 << 1 = 200
+    assertEquals(200, BytesUtils.readUnsignedVarInt(is));
+    // payload = 4
+    assertEquals(4, BytesUtils.readIntLittleEndianOnOneByte(is));
+
+    // header = 100 << 1 = 200
+    assertEquals(200, BytesUtils.readUnsignedVarInt(is));
+    // payload = 5
+    assertEquals(5, BytesUtils.readIntLittleEndianOnOneByte(is));
+
+    // end of stream
+    assertEquals(-1, is.read());
+  }
+
+  @Test
+  public void testRepeatedZeros() throws Exception {
+    // previousValue is initialized to 0
+    // make sure that repeated 0s at the beginning
+    // of the stream don't trip up the repeat count
+
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5);
+    for (int i = 0; i < 10; i++) {
+      encoder.writeInt(0);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // header = 10 << 1 = 20
+    assertEquals(20, BytesUtils.readUnsignedVarInt(is));
+    // payload = 4
+    assertEquals(0, BytesUtils.readIntLittleEndianOnOneByte(is));
+
+    // end of stream
+    assertEquals(-1, is.read());
+  }
+
+  @Test
+  public void testBitWidthZero() throws Exception {
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(0, 5);
+    for (int i = 0; i < 10; i++) {
+      encoder.writeInt(0);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // header = 10 << 1 = 20
+    assertEquals(20, BytesUtils.readUnsignedVarInt(is));
+
+    // end of stream
+    assertEquals(-1, is.read());
+  }
+
+  @Test
+  public void testBitPackingOnly() throws Exception {
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5);
+
+    for (int i = 0; i < 100; i++) {
+      encoder.writeInt(i % 3);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // header = ((104/8) << 1) | 1 = 27
+    assertEquals(27, BytesUtils.readUnsignedVarInt(is));
+
+    List<Integer> values = unpack(3, 104, is);
+
+    for (int i = 0; i < 100; i++) {
+      assertEquals(i % 3, (int) values.get(i));
+    }
+
+    // end of stream
+    assertEquals(-1, is.read());
+  }
+
+  @Test
+  public void testBitPackingOverflow() throws Exception {
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5);
+
+    for (int i = 0; i < 1000; i++) {
+      encoder.writeInt(i % 3);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // 504 is the max number of values in a bit packed run
+    // that still has a header of 1 byte
+    // header = ((504/8) << 1) | 1 = 127
+    assertEquals(127, BytesUtils.readUnsignedVarInt(is));
+    List<Integer> values = unpack(3, 504, is);
+
+    for (int i = 0; i < 504; i++) {
+      assertEquals(i % 3, (int) values.get(i));
+    }
+
+    // there should now be 496 values in another bit-packed run
+    // header = ((496/8) << 1) | 1 = 125
+    assertEquals(125, BytesUtils.readUnsignedVarInt(is));
+    values = unpack(3, 496, is);
+    for (int i = 0; i < 496; i++) {
+      assertEquals((i + 504) % 3, (int) values.get(i));
+    }
+
+    // end of stream
+    assertEquals(-1, is.read());
+  }
+
+  @Test
+  public void testTransitionFromBitPackingToRle() throws Exception {
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5);
+
+    // 5 obviously bit-packed values
+    encoder.writeInt(0);
+    encoder.writeInt(1);
+    encoder.writeInt(0);
+    encoder.writeInt(1);
+    encoder.writeInt(0);
+
+    // three repeated values, that ought to be bit-packed as well
+    encoder.writeInt(2);
+    encoder.writeInt(2);
+    encoder.writeInt(2);
+
+    // lots more repeated values, that should be rle-encoded
+    for (int i = 0; i < 100; i++) {
+      encoder.writeInt(2);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // header = ((8/8) << 1) | 1 = 3
+    assertEquals(3, BytesUtils.readUnsignedVarInt(is));
+
+    List<Integer> values = unpack(3, 8, is);
+    assertEquals(Arrays.asList(0, 1, 0, 1, 0, 2, 2, 2), values);
+
+    // header = 100 << 1 = 200
+    assertEquals(200, BytesUtils.readUnsignedVarInt(is));
+    // payload = 2
+    assertEquals(2, BytesUtils.readIntLittleEndianOnOneByte(is));
+
+    // end of stream
+    assertEquals(-1, is.read());
+  }
+
+  @Test
+  public void testPaddingZerosOnUnfinishedBitPackedRuns() throws Exception {
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(5, 5);
+    for (int i = 0; i < 9; i++) {
+      encoder.writeInt(i+1);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // header = ((16/8) << 1) | 1 = 5
+    assertEquals(5, BytesUtils.readUnsignedVarInt(is));
+
+    List<Integer> values = unpack(5, 16, is);
+
+    assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 0, 0, 0, 0, 0, 0), values);
+
+    assertEquals(-1, is.read());
+  }
+
+  @Test
+  public void testSwitchingModes() throws Exception {
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(9, 100);
+
+    // rle first
+    for (int i = 0; i < 25; i++) {
+      encoder.writeInt(17);
+    }
+
+    // bit-packing
+    for (int i = 0; i < 7; i++) {
+      encoder.writeInt(7);
+    }
+
+    encoder.writeInt(8);
+    encoder.writeInt(9);
+    encoder.writeInt(10);
+
+    // bit-packing followed by rle
+    for (int i = 0; i < 25; i++) {
+      encoder.writeInt(6);
+    }
+
+    // followed by a different rle
+    for (int i = 0; i < 8; i++) {
+      encoder.writeInt(5);
+    }
+
+    ByteArrayInputStream is = new ByteArrayInputStream(encoder.toBytes().toByteArray());
+
+    // header = 25 << 1 = 50
+    assertEquals(50, BytesUtils.readUnsignedVarInt(is));
+    // payload = 17, stored in 2 bytes
+    assertEquals(17, BytesUtils.readIntLittleEndianOnTwoBytes(is));
+
+    // header = ((16/8) << 1) | 1 = 5
+    assertEquals(5, BytesUtils.readUnsignedVarInt(is));
+    List<Integer> values = unpack(9, 16, is);
+    int v = 0;
+    for (int i = 0; i < 7; i++) {
+      assertEquals(7, (int) values.get(v));
+      v++;
+    }
+
+    assertEquals(8, (int) values.get(v++));
+    assertEquals(9, (int) values.get(v++));
+    assertEquals(10, (int) values.get(v++));
+
+    for (int i = 0; i < 6; i++) {
+      assertEquals(6, (int) values.get(v));
+      v++;
+    }
+
+    // header = 19 << 1 = 38
+    assertEquals(38, BytesUtils.readUnsignedVarInt(is));
+    // payload = 6, stored in 2 bytes
+    assertEquals(6, BytesUtils.readIntLittleEndianOnTwoBytes(is));
+
+    // header = 8 << 1  = 16
+    assertEquals(16, BytesUtils.readUnsignedVarInt(is));
+    // payload = 5, stored in 2 bytes
+    assertEquals(5, BytesUtils.readIntLittleEndianOnTwoBytes(is));
+
+    // end of stream
+    assertEquals(-1, is.read());
+  }
+
+  private static List<Integer> unpack(int bitWidth, int numValues, ByteArrayInputStream is)
+    throws Exception {
+
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int[] unpacked = new int[8];
+    byte[] next8Values = new byte[bitWidth];
+
+    List<Integer> values = new ArrayList<Integer>(numValues);
+
+    while(values.size() < numValues) {
+      for (int i = 0; i < bitWidth; i++) {
+        next8Values[i] = (byte) is.read();
+      }
+
+      packer.unpack8Values(next8Values, 0, unpacked, 0);
+
+      for (int v = 0; v < 8; v++) {
+        values.add(unpacked[v]);
+      }
+    }
+
+    return values;
+  }
+
+}
diff --git a/parquet-column/src/test/java/parquet/io/ExpectationValidatingConverter.java b/parquet-column/src/test/java/parquet/io/ExpectationValidatingConverter.java
index 623ccde..1725d79 100644
--- a/parquet-column/src/test/java/parquet/io/ExpectationValidatingConverter.java
+++ b/parquet-column/src/test/java/parquet/io/ExpectationValidatingConverter.java
@@ -17,6 +17,8 @@
 
 import static org.junit.Assert.assertEquals;
 
+import java.util.ArrayDeque;
+import java.util.Arrays;
 import java.util.Deque;
 import java.util.List;
 
@@ -43,6 +45,10 @@
     ++count;
   }
 
+  public ExpectationValidatingConverter(String[] expectations, MessageType schema) {
+    this(new ArrayDeque<String>(Arrays.asList(expectations)), schema);
+  }
+
   public ExpectationValidatingConverter(Deque<String> expectations, MessageType schema) {
     this.expectations = expectations;
     this.root = (GroupConverter)schema.convertWith(new TypeConverter<Converter>() {
diff --git a/parquet-column/src/test/java/parquet/io/PerfTest.java b/parquet-column/src/test/java/parquet/io/PerfTest.java
index 7c23c13..7424950 100644
--- a/parquet-column/src/test/java/parquet/io/PerfTest.java
+++ b/parquet-column/src/test/java/parquet/io/PerfTest.java
@@ -93,7 +93,7 @@
 
 
   private static void write(MemPageStore memPageStore) {
-    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 50*1024*1024);
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 50*1024*1024, 50*1024*1024, false);
     MessageColumnIO columnIO = newColumnFactory(schema);
 
     GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), schema);
@@ -118,20 +118,17 @@
   }
   private static void read(RecordReader<Object> recordReader, int count, MessageType schema) {
     Object[] records = new Object[count];
-    System.out.print("Trigger full gc [[[");
     System.gc();
-    System.out.print("]]]");
-    System.out.print("No gc please <<<");
+    System.out.print("no gc <");
     long t0 = System.currentTimeMillis();
     for (int i = 0; i < records.length; i++) {
       records[i] = recordReader.read();
     }
     long t1 = System.currentTimeMillis();
-    System.out.print(">>>");
+    System.out.print("> ");
     long t = t1-t0;
     float err = (float)100 * 2 / t; // (+/- 1 ms)
-    System.out.printf("                                                          " +
-    		"read %,9d recs in %,5d ms at %,9d rec/s err: %3.2f%%\n", count , t, t == 0 ? 0 : count * 1000 / t, err);
+    System.out.printf("                                            read %,9d recs in %,5d ms at %,9d rec/s err: %3.2f%%\n", count , t, t == 0 ? 0 : count * 1000 / t, err);
     if (!records[0].equals("end()")) {
       throw new RuntimeException(""+records[0]);
     }
diff --git a/parquet-column/src/test/java/parquet/io/TestColumnIO.java b/parquet-column/src/test/java/parquet/io/TestColumnIO.java
index 37f5934..c16c411 100644
--- a/parquet-column/src/test/java/parquet/io/TestColumnIO.java
+++ b/parquet-column/src/test/java/parquet/io/TestColumnIO.java
@@ -16,19 +16,24 @@
 package parquet.io;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 import static parquet.example.Paper.pr1;
 import static parquet.example.Paper.pr2;
 import static parquet.example.Paper.r1;
 import static parquet.example.Paper.r2;
 import static parquet.example.Paper.schema;
 import static parquet.example.Paper.schema2;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static parquet.schema.Type.Repetition.OPTIONAL;
+import static parquet.schema.Type.Repetition.REQUIRED;
 
-import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Deque;
+import java.util.Iterator;
 import java.util.List;
 
+import org.junit.Assert;
 import org.junit.Test;
 
 import parquet.Log;
@@ -46,8 +51,13 @@
 import parquet.io.api.Binary;
 import parquet.io.api.RecordConsumer;
 import parquet.io.api.RecordMaterializer;
+import parquet.schema.GroupType;
 import parquet.schema.MessageType;
 import parquet.schema.MessageTypeParser;
+import parquet.schema.PrimitiveType;
+import parquet.schema.PrimitiveType.PrimitiveTypeName;
+import parquet.schema.Type;
+import parquet.schema.Type.Repetition;
 
 
 public class TestColumnIO {
@@ -129,6 +139,123 @@
   }
 
   @Test
+  public void testSchemaCompatibility() {
+    MessageType schema1 = new MessageType("schema",
+        new PrimitiveType(REQUIRED, INT32, "a"));
+    MessageType schema2 = new MessageType("schema",
+        new PrimitiveType(OPTIONAL, INT32, "b"),
+        new PrimitiveType(OPTIONAL, INT32, "a")
+        );
+    MessageType schema3 = new MessageType("schema",
+        new PrimitiveType(OPTIONAL, INT32, "c"));
+
+    MemPageStore memPageStore1 = new MemPageStore();
+    MemPageStore memPageStore2 = new MemPageStore();
+    MemPageStore memPageStore3 = new MemPageStore();
+
+    SimpleGroupFactory groupFactory1 = new SimpleGroupFactory(schema1);
+    writeGroups(schema1, memPageStore1, groupFactory1.newGroup().append("a", 1));
+
+    SimpleGroupFactory groupFactory2 = new SimpleGroupFactory(schema2);
+    writeGroups(schema2, memPageStore2, groupFactory2.newGroup().append("a", 2).append("b", 3));
+
+    SimpleGroupFactory groupFactory3 = new SimpleGroupFactory(schema3);
+    writeGroups(schema3, memPageStore3, groupFactory3.newGroup().append("c", 4));
+
+    {
+      List<Group> groups1 = new ArrayList<Group>();
+      groups1.addAll(readGroups(memPageStore1, schema1, schema2, 1));
+      groups1.addAll(readGroups(memPageStore2, schema2, schema2, 1));
+      // TODO: add once we have the support for empty projection
+//      groups1.addAll(readGroups(memPageStore3, schema3, schema2, 1));
+      Object[][] e1 = {
+          { null, 1},
+          { 3, 2},
+//          { null, null}
+      };
+      validateGroups(groups1, e1);
+    }
+
+    {
+      MessageType schema4 = new MessageType("schema",
+          new PrimitiveType(OPTIONAL, INT32, "c"),
+          new PrimitiveType(OPTIONAL, INT32, "b"),
+          new PrimitiveType(OPTIONAL, INT32, "a"));
+
+      List<Group> groups2 = new ArrayList<Group>();
+      groups2.addAll(readGroups(memPageStore1, schema1, schema4, 1));
+      groups2.addAll(readGroups(memPageStore2, schema2, schema4, 1));
+      groups2.addAll(readGroups(memPageStore3, schema3, schema4, 1));
+
+      Object[][] e2 = {
+          { null, null, 1},
+          { null, 3, 2},
+          { 4, null, null}
+      };
+      validateGroups(groups2, e2);
+    }
+
+    try {
+      MessageType schema5 = new MessageType("schema",
+          new PrimitiveType(OPTIONAL, BINARY, "c")); // Incompatible schema: different type
+      readGroups(memPageStore3, schema3, schema5, 1);
+      fail("should have thrown an incompatible schema exception");
+    } catch (ParquetDecodingException e) {
+      assertEquals("The requested schema is not compatible with the file schema. incompatible types: optional binary c != optional int32 c", e.getMessage());
+    }
+
+    try {
+      MessageType schema6 = new MessageType("schema",
+          new PrimitiveType(REQUIRED, INT32, "c")); // Incompatible schema: required when it was optional
+      readGroups(memPageStore3, schema3, schema6, 1);
+      fail("should have thrown an incompatible schema exception");
+    } catch (ParquetDecodingException e) {
+      assertEquals("The requested schema is not compatible with the file schema. incompatible types: required int32 c != optional int32 c", e.getMessage());
+    }
+
+  }
+
+  private void validateGroups(List<Group> groups1, Object[][] e1) {
+    Iterator<Group> i1 = groups1.iterator();
+    for (int i = 0; i < e1.length; i++) {
+      Object[] objects = e1[i];
+      Group next = i1.next();
+      for (int j = 0; j < objects.length; j++) {
+        Object object = objects[j];
+        if (object == null) {
+          assertEquals(0, next.getFieldRepetitionCount(j));
+        } else {
+          assertEquals("looking for r[" + i + "][" + j + "][0]=" + object, 1, next.getFieldRepetitionCount(j));
+          assertEquals(object, next.getInteger(j, 0));
+        }
+      }
+    }
+  }
+
+  private List<Group> readGroups(MemPageStore memPageStore, MessageType fileSchema, MessageType requestedSchema, int n) {
+    ColumnIOFactory columnIOFactory = new ColumnIOFactory(true);
+    MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema, fileSchema);
+    RecordReaderImplementation<Group> recordReader = getRecordReader(columnIO, requestedSchema, memPageStore);
+    List<Group> groups = new ArrayList<Group>();
+    for (int i = 0; i < n; i++) {
+      groups.add(recordReader.read());
+    }
+    return groups;
+  }
+
+  private void writeGroups(MessageType writtenSchema, MemPageStore memPageStore, Group... groups) {
+    ColumnIOFactory columnIOFactory = new ColumnIOFactory(true);
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+    MessageColumnIO columnIO = columnIOFactory.getColumnIO(writtenSchema);
+    GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), writtenSchema);
+    for (Group group : groups) {
+      groupWriter.write(group);
+    }
+    columns.flush();
+  }
+
+
+  @Test
   public void testColumnIO() {
     log(schema);
     log("r1");
@@ -137,7 +264,7 @@
     log(r2);
 
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800);
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
 
     ColumnIOFactory columnIOFactory = new ColumnIOFactory(true);
     {
@@ -155,8 +282,8 @@
       validateFSA(expectedFSA, columnIO, recordReader);
 
       List<Group> records = new ArrayList<Group>();
-      read(recordReader, schema, records);
-      read(recordReader, schema, records);
+      records.add(recordReader.read());
+      records.add(recordReader.read());
 
       int i = 0;
       for (Group record : records) {
@@ -176,8 +303,8 @@
 
       validateFSA(expectedFSA2, columnIO2, recordReader);
 
-      read(recordReader, schema2, records);
-      read(recordReader, schema2, records);
+      records.add(recordReader.read());
+      records.add(recordReader.read());
 
       int i = 0;
       for (Group record : records) {
@@ -191,30 +318,116 @@
 
   @Test
   public void testOneOfEach() {
-
-    MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800);
-
     MessageType oneOfEachSchema = MessageTypeParser.parseMessageType(oneOfEach);
-
     GroupFactory gf = new SimpleGroupFactory(oneOfEachSchema);
     Group g1 = gf.newGroup().append("a", 1l).append("b", 2).append("c", 3.0f).append("d", 4.0d).append("e", true).append("f", Binary.fromString("6"));
 
+    testSchema(oneOfEachSchema, Arrays.asList(g1));
+  }
+
+  @Test
+  public void testRequiredOfRequired() {
+
+
+    MessageType reqreqSchema = MessageTypeParser.parseMessageType(
+          "message Document {\n"
+        + "  required group foo {\n"
+        + "    required int64 bar;\n"
+        + "  }\n"
+        + "}\n");
+
+    GroupFactory gf = new SimpleGroupFactory(reqreqSchema);
+    Group g1 = gf.newGroup();
+    g1.addGroup("foo").append("bar", 2l);
+
+    testSchema(reqreqSchema, Arrays.asList(g1));
+  }
+
+  @Test
+  public void testOptionalRequiredInteraction() {
+    for (int i = 0; i < 6; i++) {
+      Type current = new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "primitive");
+      for (int j = 0; j < i; j++) {
+        current = new GroupType(Repetition.REQUIRED, "req" + j, current);
+      }
+      MessageType groupSchema = new MessageType("schema"+i, current);
+      GroupFactory gf = new SimpleGroupFactory(groupSchema);
+      List<Group> groups = new ArrayList<Group>();
+      Group root = gf.newGroup();
+      Group currentGroup = root;
+      for (int j = 0; j < i; j++) {
+        currentGroup = currentGroup.addGroup(0);
+      }
+      currentGroup.add(0, Binary.fromString("foo"));
+      groups.add(root);
+      testSchema(groupSchema, groups);
+    }
+    for (int i = 0; i < 6; i++) {
+      Type current = new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.BINARY, "primitive");
+      for (int j = 0; j < i; j++) {
+        current = new GroupType(Repetition.REQUIRED, "req" + j, current);
+      }
+      MessageType groupSchema = new MessageType("schema"+(i+6), current);
+      GroupFactory gf = new SimpleGroupFactory(groupSchema);
+      List<Group> groups = new ArrayList<Group>();
+      Group rootDefined = gf.newGroup();
+      Group rootUndefined = gf.newGroup();
+      Group currentDefinedGroup = rootDefined;
+      Group currentUndefinedGroup = rootUndefined;
+      for (int j = 0; j < i; j++) {
+        currentDefinedGroup = currentDefinedGroup.addGroup(0);
+        currentUndefinedGroup = currentUndefinedGroup.addGroup(0);
+      }
+      currentDefinedGroup.add(0, Binary.fromString("foo"));
+      groups.add(rootDefined);
+      groups.add(rootUndefined);
+      testSchema(groupSchema, groups);
+    }
+    for (int i = 0; i < 6; i++) {
+      Type current = new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.BINARY, "primitive");
+      for (int j = 0; j < 6; j++) {
+        current = new GroupType(i==j ? Repetition.OPTIONAL : Repetition.REQUIRED, "req" + j, current);
+      }
+      MessageType groupSchema = new MessageType("schema"+(i+12), current);
+      GroupFactory gf = new SimpleGroupFactory(groupSchema);
+      List<Group> groups = new ArrayList<Group>();
+      Group rootDefined = gf.newGroup();
+      Group rootUndefined = gf.newGroup();
+      Group currentDefinedGroup = rootDefined;
+      Group currentUndefinedGroup = rootUndefined;
+      for (int j = 0; j < 6; j++) {
+        currentDefinedGroup = currentDefinedGroup.addGroup(0);
+        if (i < j) {
+          currentUndefinedGroup = currentUndefinedGroup.addGroup(0);
+        }
+      }
+      currentDefinedGroup.add(0, Binary.fromString("foo"));
+      groups.add(rootDefined);
+      groups.add(rootUndefined);
+      testSchema(groupSchema, groups);
+    }
+  }
+
+  private void testSchema(MessageType messageSchema, List<Group> groups) {
+    MemPageStore memPageStore = new MemPageStore();
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+
     ColumnIOFactory columnIOFactory = new ColumnIOFactory(true);
 
-    MessageColumnIO columnIO = columnIOFactory.getColumnIO(oneOfEachSchema);
+    MessageColumnIO columnIO = columnIOFactory.getColumnIO(messageSchema);
     log(columnIO);
-    GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), oneOfEachSchema);
-    groupWriter.write(g1);
+    GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), messageSchema);
+    for (Group group : groups) {
+      groupWriter.write(group);
+    }
     columns.flush();
 
-    RecordReaderImplementation<Group> recordReader = getRecordReader(columnIO, oneOfEachSchema, memPageStore);
+    RecordReaderImplementation<Group> recordReader = getRecordReader(columnIO, messageSchema, memPageStore);
 
-    List<Group> records = new ArrayList<Group>();
-    read(recordReader, oneOfEachSchema, records);
-
-    assertEquals("deserialization does not display the same result", g1.toString(), records.get(0).toString());
-
+    for (Group group : groups) {
+      final Group got = recordReader.read();
+      assertEquals("deserialization does not display the same result", group.toString(), got.toString());
+    }
   }
 
   private RecordReaderImplementation<Group> getRecordReader(MessageColumnIO columnIO, MessageType schema, PageReadStore pageReadStore) {
@@ -245,23 +458,33 @@
   @Test
   public void testPushParser() {
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800);
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
     MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
     new GroupWriter(columnIO.getRecordWriter(columns), schema).write(r1);
     columns.flush();
 
-    final Deque<String> expectations = new ArrayDeque<String>();
-    for (String string : expectedEventsForR1) {
-      expectations.add(string);
-    }
-
-    RecordReader<Void> recordReader = columnIO.getRecordReader(memPageStore, new ExpectationValidatingConverter(expectations, schema));
+    RecordReader<Void> recordReader = columnIO.getRecordReader(memPageStore, new ExpectationValidatingConverter(expectedEventsForR1, schema));
     recordReader.read();
 
   }
 
-  public void read(RecordReader<Group> recordReader, MessageType schema, List<Group> records) {
-    records.add(recordReader.read());
+  @Test
+  public void testEmptyField() {
+    MemPageStore memPageStore = new MemPageStore();
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+    MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
+    final RecordConsumer recordWriter = columnIO.getRecordWriter(columns);
+    recordWriter.startMessage();
+    recordWriter.startField("DocId", 0);
+    recordWriter.addLong(0);
+    recordWriter.endField("DocId", 0);
+    recordWriter.startField("Links", 1);
+    try {
+      recordWriter.endField("Links", 1);
+      Assert.fail("expected exception because of empty field");
+    } catch (ParquetEncodingException e) {
+      Assert.assertEquals("empty fields are illegal, the field should be ommited completely instead", e.getMessage());
+    }
   }
 
   @Test
diff --git a/parquet-column/src/test/java/parquet/io/TestFiltered.java b/parquet-column/src/test/java/parquet/io/TestFiltered.java
new file mode 100644
index 0000000..f4d9ade
--- /dev/null
+++ b/parquet-column/src/test/java/parquet/io/TestFiltered.java
@@ -0,0 +1,148 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.io;
+
+import org.junit.Test;
+import parquet.column.impl.ColumnWriteStoreImpl;
+import parquet.column.page.mem.MemPageStore;
+import parquet.example.data.Group;
+import parquet.example.data.GroupWriter;
+import parquet.example.data.simple.convert.GroupRecordConverter;
+import parquet.io.api.RecordMaterializer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static parquet.example.Paper.r1;
+import static parquet.example.Paper.r2;
+import static parquet.example.Paper.schema;
+import static parquet.filter.AndRecordFilter.and;
+import static parquet.filter.PagedRecordFilter.page;
+import static parquet.filter.ColumnPredicates.equalTo;
+import static parquet.filter.ColumnRecordFilter.column;
+
+public class TestFiltered {
+
+  @Test
+  public void testFilterOnInteger() {
+    MemPageStore memPageStore = new MemPageStore();
+    MessageColumnIO columnIO =  new ColumnIOFactory(true).getColumnIO(schema);
+    writeTestRecords(memPageStore, columnIO, 1);
+
+    // Get first record
+    RecordMaterializer<Group> recordConverter = new GroupRecordConverter(schema);
+    RecordReaderImplementation<Group> recordReader = (RecordReaderImplementation<Group>)
+        columnIO.getRecordReader(memPageStore, recordConverter,
+            column("DocId", equalTo(10l)));
+
+    Group actual1=  recordReader.read();
+    assertNull( "There should be no more records as r2 filtered out", recordReader.read());
+    assertEquals("filtering did not return the correct record", r1.toString(), actual1.toString());
+
+    // Get second record
+    recordReader = (RecordReaderImplementation<Group>)
+        columnIO.getRecordReader(memPageStore, recordConverter,
+            column("DocId", equalTo(20l)));
+
+    Group actual2 = recordReader.read();
+    assertNull( "There should be no more records as r1 filtered out", recordReader.read());
+    assertEquals("filtering did not return the correct record", r2.toString(), actual2.toString());
+
+  }
+
+  @Test
+  public void testFilterOnString() {
+    MemPageStore memPageStore = new MemPageStore();
+    MessageColumnIO columnIO =  new ColumnIOFactory(true).getColumnIO(schema);
+    writeTestRecords(memPageStore, columnIO, 1);
+
+    // First try matching against the A url in record 1
+    RecordMaterializer<Group> recordConverter = new GroupRecordConverter(schema);
+    RecordReaderImplementation<Group> recordReader = (RecordReaderImplementation<Group>)
+        columnIO.getRecordReader(memPageStore, recordConverter,
+            column("Name.Url", equalTo("http://A")));
+
+    Group actual1 = recordReader.read();
+    assertNull( "There should be no more records as r2 filtered out", recordReader.read());
+    assertEquals("filtering did not return the correct record", r1.toString(), actual1.toString());
+
+    // Second try matching against the B url in record 1 - it should fail as we only match
+    // against the first instance of a
+    recordReader = (RecordReaderImplementation<Group>)
+        columnIO.getRecordReader(memPageStore, recordConverter,
+            column("Name.Url", equalTo("http://B")));
+
+    assertNull( "There should be no matching records", recordReader.read());
+
+    // Finally try matching against the C url in record 2
+    recordReader = (RecordReaderImplementation<Group>)
+        columnIO.getRecordReader(memPageStore, recordConverter,
+            column("Name.Url", equalTo("http://C")));
+
+    Group actual2 =  recordReader.read();
+    assertNull( "There should be no more records as r1 filtered out", recordReader.read());
+    assertEquals("filtering did not return the correct record", r2.toString(), actual2.toString());
+  }
+
+  @Test
+  public void testPaged() {
+    MemPageStore memPageStore = new MemPageStore();
+    MessageColumnIO columnIO =  new ColumnIOFactory(true).getColumnIO(schema);
+    writeTestRecords(memPageStore, columnIO, 6);
+
+    RecordMaterializer<Group> recordConverter = new GroupRecordConverter(schema);
+    RecordReaderImplementation<Group> recordReader = (RecordReaderImplementation<Group>)
+        columnIO.getRecordReader(memPageStore, recordConverter,
+                                 page(4, 4));
+
+    int count = 0;
+    while ( count < 2 ) { // starts at position 4 which should be r2
+      assertEquals("expecting record2", r2.toString(), recordReader.read().toString());
+      assertEquals("expecting record1", r1.toString(), recordReader.read().toString());
+      count++;
+    }
+    assertNull("There should be no more records", recordReader.read());
+  }
+
+  @Test
+  public void testFilteredAndPaged() {
+    MemPageStore memPageStore = new MemPageStore();
+    MessageColumnIO columnIO =  new ColumnIOFactory(true).getColumnIO(schema);
+    writeTestRecords(memPageStore, columnIO, 8);
+
+    RecordMaterializer<Group> recordConverter = new GroupRecordConverter(schema);
+    RecordReaderImplementation<Group> recordReader = (RecordReaderImplementation<Group>)
+        columnIO.getRecordReader(memPageStore, recordConverter,
+            and(column("DocId", equalTo(10l)), page(2, 4)));
+
+    int count = 0;
+    while ( count < 4 ) { // starts at position 4 which should be r2
+      assertEquals("expecting 4 x record1", r1.toString(), recordReader.read().toString());
+      count++;
+    }
+    assertNull( "There should be no more records", recordReader.read());
+  }
+
+  private void writeTestRecords(MemPageStore memPageStore, MessageColumnIO columnIO, int number) {
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 800, 800, false);
+
+    GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), schema);
+    for ( int i = 0; i < number; i++ ) {
+      groupWriter.write(r1);
+      groupWriter.write(r2);
+    }
+    columns.flush();
+  }
+}
diff --git a/parquet-column/src/test/java/parquet/io/TestRecordReaderCompiler.java b/parquet-column/src/test/java/parquet/io/TestRecordReaderCompiler.java
index 67f7fa3..988004b 100644
--- a/parquet-column/src/test/java/parquet/io/TestRecordReaderCompiler.java
+++ b/parquet-column/src/test/java/parquet/io/TestRecordReaderCompiler.java
@@ -58,7 +58,7 @@
     });
 
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStore writeStore = new ColumnWriteStoreImpl(memPageStore, 1024*1024*1);
+    ColumnWriteStore writeStore = new ColumnWriteStoreImpl(memPageStore, 1024*1024*1, 1024*1024*1, false);
     MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
     new GroupWriter(columnIO.getRecordWriter(writeStore), schema).write(r1);
     writeStore.flush();
diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml
new file mode 100644
index 0000000..71f4fe8
--- /dev/null
+++ b/parquet-common/pom.xml
@@ -0,0 +1,47 @@
+<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.0.0-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-common</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Parquet Common</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <properties>
+  </properties>
+
+  <dependencies>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/parquet-common/src/main/java/parquet/Ints.java b/parquet-common/src/main/java/parquet/Ints.java
new file mode 100644
index 0000000..1e7f51a
--- /dev/null
+++ b/parquet-common/src/main/java/parquet/Ints.java
@@ -0,0 +1,41 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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;
+
+/**
+ * Utilities for working with ints
+ *
+ * @author Alex Levenson
+ */
+public final class Ints {
+  private Ints() { }
+
+  /**
+   * Cast value to a an int, or throw an exception
+   * if there is an overflow.
+   *
+   * @param value a long to be casted to an int
+   * @return an int that is == to value
+   * @throws IllegalArgumentException if value can't be casted to an int
+   */
+  public static int checkedCast(long value) {
+    int valueI = (int) value;
+    if (valueI != value) {
+      throw new IllegalArgumentException(String.format("Overflow casting %d to an int", value));
+    }
+    return valueI;
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/Log.java b/parquet-common/src/main/java/parquet/Log.java
similarity index 98%
rename from parquet-column/src/main/java/parquet/Log.java
rename to parquet-common/src/main/java/parquet/Log.java
index 1a8a134..57080f5 100644
--- a/parquet-column/src/main/java/parquet/Log.java
+++ b/parquet-common/src/main/java/parquet/Log.java
@@ -43,7 +43,7 @@
   /**
    * this is the compile time log level
    */
-  public static final Level LEVEL = Level.INFO;
+  public static final Level LEVEL = Level.INFO; // should be INFO unless for debugging
 
   public static final boolean DEBUG = (LEVEL.intValue() <= Level.FINE.intValue());
   public static final boolean INFO = (LEVEL.intValue() <= Level.INFO.intValue());
diff --git a/parquet-column/src/main/java/parquet/ParquetRuntimeException.java b/parquet-common/src/main/java/parquet/ParquetRuntimeException.java
similarity index 100%
rename from parquet-column/src/main/java/parquet/ParquetRuntimeException.java
rename to parquet-common/src/main/java/parquet/ParquetRuntimeException.java
diff --git a/parquet-common/src/main/java/parquet/Preconditions.java b/parquet-common/src/main/java/parquet/Preconditions.java
new file mode 100644
index 0000000..0504560
--- /dev/null
+++ b/parquet-common/src/main/java/parquet/Preconditions.java
@@ -0,0 +1,50 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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;
+
+/**
+ * Utility for parameter validation
+ *
+ * @author Julien Le Dem
+ *
+ */
+public final class Preconditions {
+  private Preconditions() { }
+
+  /**
+   * @param o the param to check
+   * @param name the name of the param for the error message
+   * @return the validated o
+   * @throws NullPointerException if o is null
+   */
+  public static <T> T checkNotNull(T o, String name) throws NullPointerException {
+    if (o == null) {
+      throw new NullPointerException(name + " should not be null");
+    }
+    return o;
+  }
+
+  /**
+   * @param valid whether the argument is valid
+   * @param message error message if the argument is not valid
+   * @throws IllegalArgumentException if !valid
+   */
+  public static void checkArgument(boolean valid, String message) throws IllegalArgumentException {
+    if (!valid) {
+      throw new IllegalArgumentException(message);
+    }
+  }
+}
diff --git a/parquet-common/src/main/java/parquet/Version.java b/parquet-common/src/main/java/parquet/Version.java
new file mode 100644
index 0000000..a44724d
--- /dev/null
+++ b/parquet-common/src/main/java/parquet/Version.java
@@ -0,0 +1,100 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Enumeration;
+import java.util.Properties;
+import java.util.jar.Manifest;
+
+/**
+ * The version of the library
+ *
+ * parquet-mr version 1.0.0-SNAPSHOT (build 6cf94d29b2b7115df4de2c06e2ab4326d721eb55)
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class Version {
+  private static final Log LOG = Log.getLog(Version.class);
+  
+  public static final String VERSION_NUMBER = readVersionNumber();
+  public static final String FULL_VERSION = readFullVersion();
+
+  private static String getJarPath() {
+    final URL versionClassBaseUrl = Version.class.getResource("");
+    if (versionClassBaseUrl.getProtocol().equals("jar")) {
+      String path = versionClassBaseUrl.getPath();
+      int jarEnd = path.indexOf("!");
+      if (jarEnd != -1) {
+        String jarPath = path.substring(0, jarEnd);
+        return jarPath;
+      }
+    }
+    return null;
+  }
+
+  private static URL getResourceFromJar(String jarPath, String path) throws IOException {
+    Enumeration<URL> resources = Version.class.getClassLoader().getResources(path);
+    while (resources.hasMoreElements()) {
+      URL url = resources.nextElement();
+      if (url.getProtocol().equals("jar") && url.getPath().startsWith(jarPath)) {
+        return url;
+      }
+    }
+    return null;
+  }
+  
+  private static String readVersionNumber() {
+    String version = null;
+    try {
+      String jarPath = getJarPath();
+      if (jarPath != null) {
+        URL pomPropertiesUrl = getResourceFromJar(jarPath, "META-INF/maven/com.twitter/parquet-column/pom.properties");
+        if (pomPropertiesUrl != null) {
+          Properties properties = new Properties();
+          properties.load(pomPropertiesUrl.openStream());
+          version = properties.getProperty("version");
+        }
+      }
+    } catch (Exception e) {
+      LOG.warn("can't read from META-INF", e);
+    }
+    return version;
+  }
+
+  private static String readFullVersion() {
+    String sha = null;
+    try {
+      String jarPath = getJarPath();
+      if (jarPath != null) {
+        URL manifestUrl = getResourceFromJar(jarPath, "META-INF/MANIFEST.MF");
+        if (manifestUrl != null) {
+          Manifest manifest = new Manifest(manifestUrl.openStream());
+          sha = manifest.getMainAttributes().getValue("git-SHA-1");
+        }
+      }
+    } catch (Exception e) {
+      LOG.warn("can't read from META-INF", e);
+    }
+    return "parquet-mr" + (VERSION_NUMBER != null ? " version " + VERSION_NUMBER : "") + (sha != null ? " (build " + sha + ")" : "");
+  }
+
+  public static void main(String[] args) {
+    System.out.println(FULL_VERSION);
+  }
+}
diff --git a/parquet-common/src/main/java/parquet/bytes/BytesUtils.java b/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
new file mode 100644
index 0000000..1ef8bf3
--- /dev/null
+++ b/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
@@ -0,0 +1,200 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.bytes;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import parquet.Log;
+
+/**
+ * utility methods to deal with bytes
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class BytesUtils {
+  private static final Log LOG = Log.getLog(BytesUtils.class);
+
+  public static final Charset UTF8 = Charset.forName("UTF-8");
+
+  /**
+   * give the number of bits needed to encode an int given the max value
+   * @param bound max int that we want to encode
+   * @return the number of bits required
+   */
+  public static int getWidthFromMaxInt(int bound) {
+    return 32 - Integer.numberOfLeadingZeros(bound);
+  }
+
+  /**
+   * reads an int in little endian at the given position
+   * @param in
+   * @param offset
+   * @return
+   * @throws IOException
+   */
+  public static int readIntLittleEndian(byte[] in, int offset) throws IOException {
+    int ch4 = in[offset] & 0xff;
+    int ch3 = in[offset + 1] & 0xff;
+    int ch2 = in[offset + 2] & 0xff;
+    int ch1 = in[offset + 3] & 0xff;
+    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
+  }
+
+  public static int readIntLittleEndian(InputStream in) throws IOException {
+    // TODO: this is duplicated code in LittleEndianDataInputStream
+    int ch1 = in.read();
+    int ch2 = in.read();
+    int ch3 = in.read();
+    int ch4 = in.read();
+    if ((ch1 | ch2 | ch3 | ch4) < 0) {
+        throw new EOFException();
+    }
+    return ((ch4 << 24) + (ch3 << 16) + (ch2 << 8) + (ch1 << 0));
+  }
+
+  public static int readIntLittleEndianOnOneByte(InputStream in) throws IOException {
+      int ch1 = in.read();
+      if (ch1 < 0) {
+        throw new EOFException();
+      }
+      return ch1;
+  }
+
+  public static int readIntLittleEndianOnTwoBytes(InputStream in) throws IOException {
+      int ch1 = in.read();
+      int ch2 = in.read();
+      if ((ch1 | ch2 ) < 0) {
+          throw new EOFException();
+      }
+      return ((ch2 << 8) + (ch1 << 0));
+  }
+
+  public static int readIntLittleEndianOnThreeBytes(InputStream in) throws IOException {
+      int ch1 = in.read();
+      int ch2 = in.read();
+      int ch3 = in.read();
+      if ((ch1 | ch2 | ch3 ) < 0) {
+          throw new EOFException();
+      }
+      return ((ch3 << 16) + (ch2 << 8) + (ch1 << 0));
+  }
+
+  public static int readIntLittleEndianPaddedOnBitWidth(InputStream in, int bitWidth)
+      throws IOException {
+
+    int bytesWidth = paddedByteCountFromBits(bitWidth);
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return BytesUtils.readIntLittleEndianOnOneByte(in);
+      case 2:
+        return BytesUtils.readIntLittleEndianOnTwoBytes(in);
+      case 3:
+        return  BytesUtils.readIntLittleEndianOnThreeBytes(in);
+      case 4:
+        return BytesUtils.readIntLittleEndian(in);
+      default:
+        throw new IOException(
+          String.format("Encountered bitWidth (%d) that requires more than 4 bytes", bitWidth));
+    }
+  }
+
+  public static void writeIntLittleEndianOnOneByte(OutputStream out, int v) throws IOException {
+    out.write((v >>>  0) & 0xFF);
+  }
+
+  public static void writeIntLittleEndianOnTwoBytes(OutputStream out, int v) throws IOException {
+    out.write((v >>>  0) & 0xFF);
+    out.write((v >>>  8) & 0xFF);
+  }
+
+  public static void writeIntLittleEndianOnThreeBytes(OutputStream out, int v) throws IOException {
+    out.write((v >>>  0) & 0xFF);
+    out.write((v >>>  8) & 0xFF);
+    out.write((v >>> 16) & 0xFF);
+  }
+
+  public static void writeIntLittleEndian(OutputStream out, int v) throws IOException {
+    // TODO: this is duplicated code in LittleEndianDataOutputStream
+    out.write((v >>>  0) & 0xFF);
+    out.write((v >>>  8) & 0xFF);
+    out.write((v >>> 16) & 0xFF);
+    out.write((v >>> 24) & 0xFF);
+    if (Log.DEBUG) LOG.debug("write le int: " + v + " => "+ ((v >>>  0) & 0xFF) + " " + ((v >>>  8) & 0xFF) + " " + ((v >>> 16) & 0xFF) + " " + ((v >>> 24) & 0xFF));
+  }
+
+  /**
+   * Write a little endian int to out, using the the number of bytes required by
+   * bit width
+   */
+  public static void writeIntLittleEndianPaddedOnBitWidth(OutputStream out, int v, int bitWidth)
+      throws IOException {
+
+    int bytesWidth = paddedByteCountFromBits(bitWidth);
+    switch (bytesWidth) {
+      case 0:
+        break;
+      case 1:
+        writeIntLittleEndianOnOneByte(out, v);
+        break;
+      case 2:
+        writeIntLittleEndianOnTwoBytes(out, v);
+        break;
+      case 3:
+        writeIntLittleEndianOnThreeBytes(out, v);
+        break;
+      case 4:
+        writeIntLittleEndian(out, v);
+        break;
+      default:
+        throw new IOException(
+          String.format("Encountered value (%d) that requires more than 4 bytes", v));
+    }
+  }
+
+  public static int readUnsignedVarInt(InputStream in) throws IOException {
+    int value = 0;
+    int i = 0;
+    int b;
+    while (((b = in.read()) & 0x80) != 0) {
+      value |= (b & 0x7F) << i;
+      i += 7;
+    }
+    return value | (b << i);
+  }
+
+  public static void writeUnsignedVarInt(int value, OutputStream out) throws IOException {
+    while ((value & 0xFFFFFF80) != 0L) {
+      out.write((value & 0x7F) | 0x80);
+      value >>>= 7;
+    }
+    out.write(value & 0x7F);
+  }
+
+  /**
+   * @param bitLength a count of bits
+   * @return the corresponding byte count padded to the next byte
+   */
+  public static int paddedByteCountFromBits(int bitLength) {
+    return (bitLength + 7) / 8;
+  }
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesType.java b/parquet-common/src/test/java/parquet/TestLog.java
similarity index 69%
copy from parquet-column/src/main/java/parquet/column/values/ValuesType.java
copy to parquet-common/src/test/java/parquet/TestLog.java
index 0566303..292caca 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesType.java
+++ b/parquet-common/src/test/java/parquet/TestLog.java
@@ -13,14 +13,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.values;
+package parquet;
 
-/**
- * The different type of values we can store in columns
- *
- * @author Julien Le Dem
- *
- */
-public enum ValuesType {
-  REPETITION_LEVEL, DEFINITION_LEVEL, VALUES;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestLog {
+
+  @Test
+  public void test() {
+    // Use a compile time log level of INFO for performance
+    Assert.assertFalse("Do not merge in log level DEBUG", Log.DEBUG);
+  }
 }
diff --git a/parquet-column/src/test/java/parquet/bytes/TestBytesUtil.java b/parquet-common/src/test/java/parquet/bytes/TestBytesUtil.java
similarity index 100%
rename from parquet-column/src/test/java/parquet/bytes/TestBytesUtil.java
rename to parquet-common/src/test/java/parquet/bytes/TestBytesUtil.java
diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml
new file mode 100644
index 0000000..3d2709d
--- /dev/null
+++ b/parquet-encoding/pom.xml
@@ -0,0 +1,83 @@
+<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.0.0-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-encoding</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Parquet Encodings</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <properties>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-generator</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+      <version>1.7</version>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.2.1</version>
+        <executions>
+          <execution>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>java</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <mainClass>parquet.encoding.Generator</mainClass>          
+          <arguments>
+            <argument>${basedir}/target/generated-src</argument>
+          </arguments>
+          <sourceRoot>${basedir}/target/generated-src</sourceRoot>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/parquet-column/src/main/java/parquet/bytes/BytesInput.java b/parquet-encoding/src/main/java/parquet/bytes/BytesInput.java
similarity index 72%
rename from parquet-column/src/main/java/parquet/bytes/BytesInput.java
rename to parquet-encoding/src/main/java/parquet/bytes/BytesInput.java
index 2331b89..3298a94 100644
--- a/parquet-column/src/main/java/parquet/bytes/BytesInput.java
+++ b/parquet-encoding/src/main/java/parquet/bytes/BytesInput.java
@@ -20,13 +20,19 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.List;
 
 import parquet.Log;
 
 
 /**
- *
- * A source of bytes capable of writing itself to an output
+ * A source of bytes capable of writing itself to an output.
+ * A BytesInput should be consumed right away.
+ * It is not a container.
+ * For example if it is referring to a stream,
+ * subsequent BytesInput reads from the stream will be incorrect
+ * if the previous has not been consumed.
  *
  * @author Julien Le Dem
  *
@@ -38,10 +44,19 @@
 
   /**
    * logically concatenate the provided inputs
-   * @param inputs the concatenated inputs
+   * @param inputs the inputs to concatenate
    * @return a concatenated input
    */
-  public static BytesInput fromSequence(BytesInput... inputs) {
+  public static BytesInput concat(BytesInput... inputs) {
+    return new SequenceBytesIn(Arrays.asList(inputs));
+  }
+
+  /**
+   * logically concatenate the provided inputs
+   * @param inputs the inputs to concatenate
+   * @return a concatenated input
+   */
+  public static BytesInput concat(List<BytesInput> inputs) {
     return new SequenceBytesIn(inputs);
   }
 
@@ -61,11 +76,15 @@
    */
   public static BytesInput from(byte[] in) {
     if (DEBUG) LOG.debug("BytesInput from array of " + in.length + " bytes");
-    return new ByteArrayBytesInput(in);
+    return new ByteArrayBytesInput(in, 0 , in.length);
+  }
+
+  public static BytesInput from(byte[] in, int offset, int length) {
+    if (DEBUG) LOG.debug("BytesInput from array of " + length + " bytes");
+    return new ByteArrayBytesInput(in, offset, length);
   }
 
   /**
-   *
    * @param intValue the int to write
    * @return a BytesInput that will write 4 bytes in little endian
    */
@@ -74,12 +93,19 @@
   }
 
   /**
-   *
    * @param arrayOut
    * @return a BytesInput that will write the content of the buffer
    */
-  public static BytesInput from(ByteArrayOutputStream arrayOut) {
-    return new BAOSBytesInput(arrayOut);
+  public static BytesInput from(CapacityByteArrayOutputStream arrayOut) {
+    return new CapacityBAOSBytesInput(arrayOut);
+  }
+
+  /**
+   * @param arrayOut
+   * @return a BytesInput that will write the content of the buffer
+   */
+  public static BytesInput from(ByteArrayOutputStream baos) {
+    return new BAOSBytesInput(baos);
   }
 
   /**
@@ -149,9 +175,14 @@
     public void writeAllTo(OutputStream out) throws IOException {
       if (DEBUG) LOG.debug("write All "+ byteCount + " bytes");
       // TODO: more efficient
+      out.write(this.toByteArray());
+    }
+
+    public byte[] toByteArray() throws IOException {
+      if (DEBUG) LOG.debug("read all "+ byteCount + " bytes");
       byte[] buf = new byte[byteCount];
       new DataInputStream(in).readFully(buf);
-      out.write(buf);
+      return buf;
     }
 
     @Override
@@ -164,10 +195,10 @@
   private static class SequenceBytesIn extends BytesInput {
     private static final Log LOG = Log.getLog(BytesInput.SequenceBytesIn.class);
 
-    public final BytesInput[] inputs;
+    private final List<BytesInput> inputs;
     private final long size;
 
-    private SequenceBytesIn(BytesInput[] inputs) {
+    private SequenceBytesIn(List<BytesInput> inputs) {
       this.inputs = inputs;
       long total = 0;
       for (BytesInput input : inputs) {
@@ -176,6 +207,7 @@
       this.size = total;
     }
 
+    @SuppressWarnings("unused")
     @Override
     public void writeAllTo(OutputStream out) throws IOException {
       for (BytesInput input : inputs) {
@@ -226,6 +258,26 @@
 
   }
 
+  private static class CapacityBAOSBytesInput extends BytesInput {
+
+    private final CapacityByteArrayOutputStream arrayOut;
+
+    private CapacityBAOSBytesInput(CapacityByteArrayOutputStream arrayOut) {
+      this.arrayOut = arrayOut;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      arrayOut.writeTo(out);
+    }
+
+    @Override
+    public long size() {
+      return arrayOut.size();
+    }
+
+  }
+
   private static class BAOSBytesInput extends BytesInput {
 
     private final ByteArrayOutputStream arrayOut;
@@ -249,20 +301,25 @@
   private static class ByteArrayBytesInput extends BytesInput {
 
     private final byte[] in;
+    private final int offset;
+    private final int length;
 
-    private ByteArrayBytesInput(byte[] in) {
+    private ByteArrayBytesInput(byte[] in, int offset, int length) {
       this.in = in;
+      this.offset = offset;
+      this.length = length;
     }
 
     @Override
     public void writeAllTo(OutputStream out) throws IOException {
-      out.write(in);
+      out.write(in, offset, length);
     }
 
     @Override
     public long size() {
-      return in.length;
+      return length;
     }
 
   }
+
 }
diff --git a/parquet-encoding/src/main/java/parquet/bytes/CapacityByteArrayOutputStream.java b/parquet-encoding/src/main/java/parquet/bytes/CapacityByteArrayOutputStream.java
new file mode 100644
index 0000000..96258af
--- /dev/null
+++ b/parquet-encoding/src/main/java/parquet/bytes/CapacityByteArrayOutputStream.java
@@ -0,0 +1,245 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.bytes;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import parquet.Log;
+import parquet.Preconditions;
+
+/**
+ * functionality of ByteArrayOutputStream without the memory and copy overhead
+ *
+ * It will linearly create a new slab of the initial size when needed (instead of creating a new buffer and copying the data).
+ * After 10 slabs their size will increase exponentially (similar to {@link ByteArrayOutputStream} behavior) by making the new slab size the size of the existing data.
+ *
+ * When reusing a buffer it will adjust the slab size based on the previous data size ({@link CapacityByteArrayOutputStream#reset()})
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class CapacityByteArrayOutputStream extends OutputStream {
+  private static final Log LOG = Log.getLog(CapacityByteArrayOutputStream.class);
+
+  private static final int MINIMUM_SLAB_SIZE = 64 * 1024;
+  private static final int EXPONENTIAL_SLAB_SIZE_THRESHOLD = 10;
+
+  private int slabSize;
+  private List<byte[]> slabs = new ArrayList<byte[]>();
+  private byte[] currentSlab;
+  private int capacity;
+  private int currentSlabIndex;
+  private int currentSlabPosition;
+  private int size;
+
+  /**
+   * @param initialSize the initialSize of the buffer (also slab size)
+   */
+  public CapacityByteArrayOutputStream(int initialSize) {
+    Preconditions.checkArgument(initialSize > 0, "initialSize must be > 0");
+    initSlabs(initialSize);
+  }
+
+  private void initSlabs(int initialSize) {
+    if (Log.DEBUG) LOG.debug(String.format("initial slab of size %d", initialSize));
+    this.slabSize = initialSize;
+    this.slabs.clear();
+    this.capacity = initialSize;
+    this.currentSlab = new byte[slabSize];
+    this.slabs.add(currentSlab);
+    this.currentSlabIndex = 0;
+    this.currentSlabPosition = 0;
+    this.size = 0;
+  }
+
+  private void addSlab(int minimumSize) {
+    this.currentSlabIndex += 1;
+    if (currentSlabIndex < this.slabs.size()) {
+      // reuse existing slab
+      this.currentSlab = this.slabs.get(currentSlabIndex);
+      if (Log.DEBUG) LOG.debug(String.format("reusing slab of size %d", currentSlab.length));
+      if (currentSlab.length < minimumSize) {
+        if (Log.DEBUG) LOG.debug(String.format("slab size %,d too small for value of size %,d. replacing slab", currentSlab.length, minimumSize));
+        byte[] newSlab = new byte[minimumSize];
+        capacity += minimumSize - currentSlab.length;
+        this.currentSlab = newSlab;
+        this.slabs.set(currentSlabIndex, newSlab);
+      }
+    } else {
+      if (currentSlabIndex > EXPONENTIAL_SLAB_SIZE_THRESHOLD) {
+        // make slabs bigger in case we are creating too many of them
+        // double slab size every time.
+        this.slabSize = size;
+        if (Log.DEBUG) LOG.debug(String.format("used %d slabs, new slab size %d", currentSlabIndex, slabSize));
+      }
+      if (slabSize < minimumSize) {
+        if (Log.DEBUG) LOG.debug(String.format("slab size %,d too small for value of size %,d. Bumping up slab size", slabSize, minimumSize));
+        this.slabSize = minimumSize;
+      }
+      if (Log.DEBUG) LOG.debug(String.format("new slab of size %d", slabSize));
+      this.currentSlab = new byte[slabSize];
+      this.slabs.add(currentSlab);
+      this.capacity += slabSize;
+    }
+    this.currentSlabPosition = 0;
+  }
+
+  @Override
+  public void write(int b) {
+    if (currentSlabPosition == currentSlab.length) {
+      addSlab(1);
+    }
+    currentSlab[currentSlabPosition] = (byte) b;
+    currentSlabPosition += 1;
+    size += 1;
+  }
+
+  @Override
+  public void write(byte b[], int off, int len) {
+    if ((off < 0) || (off > b.length) || (len < 0) ||
+        ((off + len) - b.length > 0)) {
+      throw new IndexOutOfBoundsException();
+    }
+    if (currentSlabPosition + len >= currentSlab.length) {
+      final int length1 = currentSlab.length - currentSlabPosition;
+      System.arraycopy(b, off, currentSlab, currentSlabPosition, length1);
+      final int length2 = len - length1;
+      addSlab(length2);
+      System.arraycopy(b, off + length1, currentSlab, currentSlabPosition, length2);
+      currentSlabPosition = length2;
+    } else {
+      System.arraycopy(b, off, currentSlab, currentSlabPosition, len);
+      currentSlabPosition += len;
+    }
+    size += len;
+  }
+
+  /**
+   * Writes the complete contents of this buffer to the specified output stream argument. the output
+   * stream's write method <code>out.write(slab, 0, slab.length)</code>) will be called once per slab.
+   *
+   * @param      out   the output stream to which to write the data.
+   * @exception  IOException  if an I/O error occurs.
+   */
+  public void writeTo(OutputStream out) throws IOException {
+    for (int i = 0; i < currentSlabIndex; i++) {
+      final byte[] slab = slabs.get(i);
+      out.write(slab, 0, slab.length);
+    }
+    out.write(currentSlab, 0, currentSlabPosition);
+  }
+
+  /**
+   * @return the size of the allocated buffer
+   */
+  public int getCapacity() {
+    return capacity;
+  }
+
+  /**
+   * When re-using an instance with reset, it will adjust slab size based on previous data size.
+   * The intent is to reuse the same instance for the same type of data (for example, the same column).
+   * The assumption is that the size in the buffer will be consistent. Otherwise we fall back to exponentialy double the slab size.
+   * <ul>
+   * <li>if we used less than half of the first slab (and it is above the minimum slab size), we will make the slab size smaller.
+   * <li>if we used more than the slab count threshold (10), we will re-adjust the slab size.
+   * </ul>
+   * if re-adjusting the slab size we will make it 1/5th of the previous used size in the buffer so that overhead of extra memory allocation is about 20%
+   * If we used less than the available slabs we free up the unused ones to reduce memory overhead.
+   */
+  public void reset() {
+    // heuristics to adjust slab size
+    if (
+        // if we have only one slab, make sure it is not way too big (more than twice what we need). Except if the slab is already small
+        (currentSlabIndex == 0 && currentSlabPosition < currentSlab.length / 2 && currentSlab.length > MINIMUM_SLAB_SIZE)
+        ||
+        // we want to avoid generating too many slabs.
+        (currentSlabIndex > EXPONENTIAL_SLAB_SIZE_THRESHOLD)
+        ){
+      // readjust slab size
+      initSlabs(Math.max(size / 5, MINIMUM_SLAB_SIZE)); // should make overhead to about 20% without incurring many slabs
+      if (Log.DEBUG) LOG.debug(String.format("used %d slabs, new slab size %d", currentSlabIndex + 1, slabSize));
+    } else if (currentSlabIndex < slabs.size() - 1) {
+      // free up the slabs that we are not using. We want to minimize overhead
+      this.slabs = new ArrayList<byte[]>(slabs.subList(0, currentSlabIndex + 1));
+      this.capacity = 0;
+      for (byte[] slab : slabs) {
+        capacity += slab.length;
+      }
+    }
+    this.currentSlabIndex = 0;
+    this.currentSlabPosition = 0;
+    this.currentSlab = slabs.get(currentSlabIndex);
+    this.size = 0;
+  }
+
+  /**
+   * @return the size of the buffered data
+   */
+  public long size() {
+    return size;
+  }
+
+  /**
+   * @return the index of the last value being written to this stream, which
+   * can be passed to {@link #setByte(long, byte)} in order to change it
+   */
+  public long getCurrentIndex() {
+    Preconditions.checkArgument(size > 0, "This is an empty stream");
+    return size - 1;
+  }
+
+  /**
+   * Replace the byte stored at position index in this stream with value
+   *
+   * @param index which byte to replace
+   * @param value the value to replace it with
+   */
+  public void setByte(long index, byte value) {
+    Preconditions.checkArgument(index < size,
+      "Index: " + index + " is >= the current size of: " + size);
+
+    long seen = 0;
+    for (int i = 0; i <=currentSlabIndex; i++) {
+      byte[] slab = slabs.get(i);
+      if (index < seen + slab.length) {
+        // ok found index
+        slab[(int)(index-seen)] = value;
+        break;
+      }
+      seen += slab.length;
+    }
+  }
+
+  /**
+   * @param prefix  a prefix to be used for every new line in the string
+   * @return a text representation of the memory usage of this structure
+   */
+  public String memUsageString(String prefix) {
+    return String.format("%s %s %d slabs, %,d bytes", prefix, getClass().getSimpleName(), slabs.size(), getCapacity());
+  }
+
+  /**
+   * @return the total count of allocated slabs
+   */
+  int getSlabCount() {
+    return slabs.size();
+  }
+}
\ No newline at end of file
diff --git a/parquet-column/src/main/java/parquet/bytes/LittleEndianDataInputStream.java b/parquet-encoding/src/main/java/parquet/bytes/LittleEndianDataInputStream.java
similarity index 100%
rename from parquet-column/src/main/java/parquet/bytes/LittleEndianDataInputStream.java
rename to parquet-encoding/src/main/java/parquet/bytes/LittleEndianDataInputStream.java
diff --git a/parquet-column/src/main/java/parquet/bytes/LittleEndianDataOutputStream.java b/parquet-encoding/src/main/java/parquet/bytes/LittleEndianDataOutputStream.java
similarity index 100%
rename from parquet-column/src/main/java/parquet/bytes/LittleEndianDataOutputStream.java
rename to parquet-encoding/src/main/java/parquet/bytes/LittleEndianDataOutputStream.java
diff --git a/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPacking.java b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/BitPacking.java
similarity index 99%
rename from parquet-column/src/main/java/parquet/column/values/bitpacking/BitPacking.java
rename to parquet-encoding/src/main/java/parquet/column/values/bitpacking/BitPacking.java
index 07f7993..b630fb4 100644
--- a/parquet-column/src/main/java/parquet/column/values/bitpacking/BitPacking.java
+++ b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/BitPacking.java
@@ -19,6 +19,7 @@
 import java.io.InputStream;
 import java.io.OutputStream;
 
+import parquet.bytes.BytesUtils;
 import parquet.column.values.bitpacking.BitPacking.BitPackingReader;
 import parquet.column.values.bitpacking.BitPacking.BitPackingWriter;
 
@@ -157,7 +158,7 @@
 abstract class BaseBitPackingReader extends BitPackingReader {
 
   int alignToBytes(int bitsCount) {
-    return bitsCount / 8 + (bitsCount % 8 == 0 ? 0 : 1);
+    return BytesUtils.paddedByteCountFromBits(bitsCount);
   }
 
 }
diff --git a/parquet-encoding/src/main/java/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java
new file mode 100644
index 0000000..bcede62
--- /dev/null
+++ b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/ByteBasedBitPackingEncoder.java
@@ -0,0 +1,127 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+import static parquet.Log.DEBUG;
+import static parquet.bytes.BytesInput.concat;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import parquet.Log;
+import parquet.bytes.BytesInput;
+import parquet.bytes.BytesUtils;
+
+/**
+ * Uses the generated Byte based bit packing to write ints into a BytesInput
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class ByteBasedBitPackingEncoder {
+  private static final Log LOG = Log.getLog(ByteBasedBitPackingEncoder.class);
+
+  private static final int VALUES_WRITTEN_AT_A_TIME = 8;
+
+  private final int bitWidth;
+  private final BytePacker packer;
+  private final int[] input = new int[VALUES_WRITTEN_AT_A_TIME];
+  private final int slabSize;
+  private int inputSize;
+  private byte[] packed;
+  private int packedPosition;
+  private final List<BytesInput> slabs = new ArrayList<BytesInput>();
+  private int totalValues;
+
+  /**
+   * @param bitWidth the number of bits used to encode an int
+   */
+  public ByteBasedBitPackingEncoder(int bitWidth, Packer packer) {
+    this.bitWidth = bitWidth;
+    this.inputSize = 0;
+    // must be a multiple of bitWidth
+    this.slabSize = bitWidth * 64 * 1024;
+    initPackedSlab();
+    this.packer = packer.newBytePacker(bitWidth);
+  }
+
+  /**
+   * writes an int using the requested number of bits.
+   * accepts only value < 2^bitWidth
+   * @param value the value to write
+   * @throws IOException
+   */
+  public void writeInt(int value) throws IOException {
+    input[inputSize] = value;
+    ++ inputSize;
+    if (inputSize == VALUES_WRITTEN_AT_A_TIME) {
+      pack();
+      if (packedPosition == slabSize) {
+        slabs.add(BytesInput.from(packed));
+        initPackedSlab();
+      }
+    }
+  }
+
+  private void pack() {
+    packer.pack8Values(input, 0, packed, packedPosition);
+    packedPosition += bitWidth;
+    totalValues += inputSize;
+    inputSize = 0;
+  }
+
+  private void initPackedSlab() {
+    packed = new byte[slabSize];
+    packedPosition = 0;
+  }
+
+  /**
+   * @return the bytes representing the packed values
+   * @throws IOException
+   */
+  public BytesInput toBytes() throws IOException {
+    int packedByteLength = packedPosition + BytesUtils.paddedByteCountFromBits(inputSize * bitWidth);
+
+    if (DEBUG) LOG.debug("writing " + (slabs.size() * slabSize + packedByteLength) + " bytes");
+    if (inputSize > 0) {
+      for (int i = inputSize; i < input.length; i++) {
+        input[i] = 0;
+      }
+      pack();
+    }
+    return concat(concat(slabs), BytesInput.from(packed, 0, packedByteLength));
+  }
+
+  /**
+   * @return size of the data as it would be written
+   */
+  public long getBufferSize() {
+    return BytesUtils.paddedByteCountFromBits(totalValues * bitWidth);
+  }
+
+  /**
+   * @return total memory allocated
+   */
+  public long getAllocatedSize() {
+    return (slabs.size() * slabSize) + packed.length + input.length * 4;
+  }
+
+  public String memUsageString(String prefix) {
+    return String.format("%s ByteBitPacking %d slabs, %d bytes", prefix, slabs.size(), getAllocatedSize());
+  }
+
+}
diff --git a/parquet-encoding/src/main/java/parquet/column/values/bitpacking/BytePacker.java b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/BytePacker.java
new file mode 100644
index 0000000..ad35d2c
--- /dev/null
+++ b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/BytePacker.java
@@ -0,0 +1,83 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+/**
+ * Packs and unpacks into bytes
+ *
+ * packing unpacking treats:
+ *  - n values at a time (with n % 8 == 0)
+ *  - bitWidth * (n/8) bytes at a time.
+ *
+ * @author Julien Le Dem
+ *
+ */
+public abstract class BytePacker {
+
+  private final int bitWidth;
+
+  BytePacker(int bitWidth) {
+    this.bitWidth = bitWidth;
+  }
+
+  /**
+   * @return the width in bits used for encoding, also how many bytes are packed/unpacked at a time by pack8Values/unpack8Values
+   */
+  public final int getBitWidth() {
+    return bitWidth;
+  }
+
+  /**
+   * pack 8 values from input at inPos into bitWidth bytes in output at outPos.
+   * nextPosition: inPos += 8; outPos += getBitWidth()
+   * @param input the input values
+   * @param inPos where to read from in input
+   * @param output the output bytes
+   * @param outPos where to write to in output
+   */
+  public abstract void pack8Values(final int[] input, final int inPos, final byte[] output, final int outPos);
+
+  /**
+   * pack 32 values from input at inPos into bitWidth * 4 bytes in output at outPos.
+   * nextPosition: inPos += 32; outPos += getBitWidth() * 4
+   * @param input the input values
+   * @param inPos where to read from in input
+   * @param output the output bytes
+   * @param outPos where to write to in output
+   */
+  public abstract void pack32Values(int[] input, int inPos, byte[] output, int outPos);
+
+  /**
+   * unpack bitWidth bytes from input at inPos into 8 values in output at outPos.
+   * nextPosition: inPos += getBitWidth(); outPos += 8
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public abstract void unpack8Values(final byte[] input, final int inPos, final int[] output, final int outPos);
+
+  /**
+   * unpack bitWidth * 4 bytes from input at inPos into 32 values in output at outPos.
+   * nextPosition: inPos += getBitWidth() * 4; outPos += 32
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public abstract void unpack32Values(byte[] input, int inPos, int[] output, int outPos);
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesType.java b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/BytePackerFactory.java
similarity index 74%
copy from parquet-column/src/main/java/parquet/column/values/ValuesType.java
copy to parquet-encoding/src/main/java/parquet/column/values/bitpacking/BytePackerFactory.java
index 0566303..95515c8 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesType.java
+++ b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/BytePackerFactory.java
@@ -13,14 +13,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.values;
+package parquet.column.values.bitpacking;
 
-/**
- * The different type of values we can store in columns
- *
- * @author Julien Le Dem
- *
- */
-public enum ValuesType {
-  REPETITION_LEVEL, DEFINITION_LEVEL, VALUES;
+public interface BytePackerFactory {
+
+  BytePacker newBytePacker(int width);
+
 }
diff --git a/parquet-encoding/src/main/java/parquet/column/values/bitpacking/IntPacker.java b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/IntPacker.java
new file mode 100644
index 0000000..2223fd4
--- /dev/null
+++ b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/IntPacker.java
@@ -0,0 +1,63 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+/**
+ * Packs and unpacks into ints
+ *
+ * packing unpacking treats:
+ *  - 32 values at a time
+ *  - bitWidth ints at a time.
+ *
+ * @author Julien Le Dem
+ *
+ */
+public abstract class IntPacker {
+
+  private final int bitWidth;
+
+  IntPacker(int bitWidth) {
+    this.bitWidth = bitWidth;
+  }
+
+  /**
+   * @return the width in bits used for encoding, also how many ints are packed/unpacked at a time
+   */
+  public final int getBitWidth() {
+    return bitWidth;
+  }
+
+  /**
+   * pack 32 values from input at inPos into bitWidth ints in output at outPos.
+   * nextPosition: inPos += 32; outPos += getBitWidth()
+   * @param input the input values
+   * @param inPos where to read from in input
+   * @param output the output ints
+   * @param outPos where to write to in output
+   */
+  public abstract void pack32Values(int[] input, int inPos, int[] output, int outPos);
+
+  /**
+   * unpack bitWidth ints from input at inPos into 32 values in output at outPos.
+   * nextPosition: inPos += getBitWidth(); outPos += 32
+   * @param input the input int
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public abstract void unpack32Values(int[] input, int inPos, int[] output, int outPos);
+
+}
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesType.java b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/IntPackerFactory.java
similarity index 74%
copy from parquet-column/src/main/java/parquet/column/values/ValuesType.java
copy to parquet-encoding/src/main/java/parquet/column/values/bitpacking/IntPackerFactory.java
index 0566303..e41f186 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesType.java
+++ b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/IntPackerFactory.java
@@ -13,14 +13,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.values;
+package parquet.column.values.bitpacking;
 
-/**
- * The different type of values we can store in columns
- *
- * @author Julien Le Dem
- *
- */
-public enum ValuesType {
-  REPETITION_LEVEL, DEFINITION_LEVEL, VALUES;
+public interface IntPackerFactory {
+
+  IntPacker newIntPacker(int width);
+
 }
diff --git a/parquet-encoding/src/main/java/parquet/column/values/bitpacking/Packer.java b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/Packer.java
new file mode 100644
index 0000000..af8394f
--- /dev/null
+++ b/parquet-encoding/src/main/java/parquet/column/values/bitpacking/Packer.java
@@ -0,0 +1,96 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+/**
+ * Factory for packing implementations
+ *
+ * @author Julien Le Dem
+ *
+ */
+public enum Packer {
+
+  /**
+   * packers who fill the Least Significant Bit First
+   * int and byte packer have the same result on Big Endian architectures
+   */
+  BIG_ENDIAN {
+    @Override
+    public IntPacker newIntPacker(int width) {
+      return beIntPackerFactory.newIntPacker(width);
+    }
+    @Override
+    public BytePacker newBytePacker(int width) {
+      return beBytePackerFactory.newBytePacker(width);
+    }
+  },
+
+  /**
+   * packers who fill the Most Significant Bit first
+   * int and byte packer have the same result on Little Endian architectures
+   */
+  LITTLE_ENDIAN {
+    @Override
+    public IntPacker newIntPacker(int width) {
+      return leIntPackerFactory.newIntPacker(width);
+    }
+    @Override
+    public BytePacker newBytePacker(int width) {
+      return leBytePackerFactory.newBytePacker(width);
+    }
+  };
+
+  private static IntPackerFactory getIntPackerFactory(String name) {
+    return (IntPackerFactory)getStaticField("parquet.column.values.bitpacking." + name, "factory");
+  }
+
+  private static BytePackerFactory getBytePackerFactory(String name) {
+    return (BytePackerFactory)getStaticField("parquet.column.values.bitpacking." + name, "factory");
+  }
+
+  private static Object getStaticField(String className, String fieldName) {
+    try {
+      return Class.forName(className).getField(fieldName).get(null);
+    } catch (IllegalArgumentException e) {
+      throw new RuntimeException(e);
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e);
+    } catch (NoSuchFieldException e) {
+      throw new RuntimeException(e);
+    } catch (SecurityException e) {
+      throw new RuntimeException(e);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  static BytePackerFactory beBytePackerFactory = getBytePackerFactory("ByteBitPackingBE");
+  static IntPackerFactory beIntPackerFactory = getIntPackerFactory("LemireBitPackingBE");
+  static BytePackerFactory leBytePackerFactory = getBytePackerFactory("ByteBitPackingLE");
+  static IntPackerFactory leIntPackerFactory = getIntPackerFactory("LemireBitPackingLE");
+
+  /**
+   * @param width the width in bits of the packed values
+   * @return an int based packer
+   */
+  public abstract IntPacker newIntPacker(int width);
+
+  /**
+   * @param width the width in bits of the packed values
+   * @return a byte based packer
+   */
+  public abstract BytePacker newBytePacker(int width);
+}
\ No newline at end of file
diff --git a/parquet-encoding/src/test/java/parquet/bytes/TestCapacityByteArrayOutputStream.java b/parquet-encoding/src/test/java/parquet/bytes/TestCapacityByteArrayOutputStream.java
new file mode 100644
index 0000000..dd10cd5
--- /dev/null
+++ b/parquet-encoding/src/test/java/parquet/bytes/TestCapacityByteArrayOutputStream.java
@@ -0,0 +1,235 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.bytes;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.junit.Test;
+
+public class TestCapacityByteArrayOutputStream {
+
+  @Test
+  public void testWrite() throws Throwable {
+    CapacityByteArrayOutputStream capacityByteArrayOutputStream = new CapacityByteArrayOutputStream(10);
+    final int expectedSize = 54;
+    for (int i = 0; i < expectedSize; i++) {
+      capacityByteArrayOutputStream.write(i);
+      assertEquals(i + 1, capacityByteArrayOutputStream.size());
+    }
+    validate(capacityByteArrayOutputStream, expectedSize);
+  }
+
+  @Test
+  public void testWriteArray() throws Throwable {
+    CapacityByteArrayOutputStream capacityByteArrayOutputStream = new CapacityByteArrayOutputStream(10);
+    int v = 23;
+    writeArraysOf3(capacityByteArrayOutputStream, v);
+    validate(capacityByteArrayOutputStream, v * 3);
+  }
+
+  @Test
+  public void testWriteArrayAndInt() throws Throwable {
+    CapacityByteArrayOutputStream capacityByteArrayOutputStream = new CapacityByteArrayOutputStream(10);
+    for (int i = 0; i < 23; i++) {
+      byte[] toWrite = { (byte)(i * 3), (byte)(i * 3 + 1)};
+      capacityByteArrayOutputStream.write(toWrite);
+      capacityByteArrayOutputStream.write((byte)(i * 3 + 2));
+      assertEquals((i + 1) * 3, capacityByteArrayOutputStream.size());
+    }
+    validate(capacityByteArrayOutputStream, 23 * 3);
+
+  }
+
+  @Test
+  public void testReset() throws Throwable {
+    CapacityByteArrayOutputStream capacityByteArrayOutputStream = new CapacityByteArrayOutputStream(10);
+    for (int i = 0; i < 54; i++) {
+      capacityByteArrayOutputStream.write(i);
+      assertEquals(i + 1, capacityByteArrayOutputStream.size());
+    }
+    capacityByteArrayOutputStream.reset();
+    for (int i = 0; i < 54; i++) {
+      capacityByteArrayOutputStream.write(54 + i);
+      assertEquals(i + 1, capacityByteArrayOutputStream.size());
+    }
+    final byte[] byteArray = BytesInput.from(capacityByteArrayOutputStream).toByteArray();
+    assertEquals(54, byteArray.length);
+    for (int i = 0; i < 54; i++) {
+      assertEquals(i + " in " + Arrays.toString(byteArray) ,54 + i, byteArray[i]);
+    }
+  }
+
+  @Test
+  public void testWriteArrayBiggerThanSlab() throws Throwable {
+    CapacityByteArrayOutputStream capacityByteArrayOutputStream = new CapacityByteArrayOutputStream(10);
+    int v = 23;
+    writeArraysOf3(capacityByteArrayOutputStream, v);
+    int n = v * 3;
+    byte[] toWrite = { // bigger than 2 slabs of size of 10
+        (byte)n, (byte)(n + 1), (byte)(n + 2), (byte)(n + 3), (byte)(n + 4), (byte)(n + 5),
+        (byte)(n + 6), (byte)(n + 7), (byte)(n + 8), (byte)(n + 9), (byte)(n + 10),
+        (byte)(n + 11), (byte)(n + 12), (byte)(n + 13), (byte)(n + 14), (byte)(n + 15),
+        (byte)(n + 16), (byte)(n + 17), (byte)(n + 18), (byte)(n + 19), (byte)(n + 20)};
+    capacityByteArrayOutputStream.write(toWrite);
+    n = n + toWrite.length;
+    assertEquals(n, capacityByteArrayOutputStream.size());
+    validate(capacityByteArrayOutputStream, n);
+    capacityByteArrayOutputStream.reset();
+    // check it works after reset too
+    capacityByteArrayOutputStream.write(toWrite);
+    assertEquals(toWrite.length, capacityByteArrayOutputStream.size());
+    byte[] byteArray = BytesInput.from(capacityByteArrayOutputStream).toByteArray();
+    assertEquals(toWrite.length, byteArray.length);
+    for (int i = 0; i < toWrite.length; i++) {
+      assertEquals(toWrite[i], byteArray[i]);
+    }
+  }
+
+  @Test
+  public void testWriteArrayManySlabs() throws Throwable {
+    CapacityByteArrayOutputStream capacityByteArrayOutputStream = new CapacityByteArrayOutputStream(10);
+    int it = 500;
+    int v = 23;
+    for (int j = 0; j < it; j++) {
+      for (int i = 0; i < v; i++) {
+        byte[] toWrite = { (byte)(i * 3), (byte)(i * 3 + 1), (byte)(i * 3 + 2)};
+        capacityByteArrayOutputStream.write(toWrite);
+        assertEquals((i + 1) * 3 + v * 3 * j, capacityByteArrayOutputStream.size());
+      }
+    }
+    byte[] byteArray = BytesInput.from(capacityByteArrayOutputStream).toByteArray();
+    assertEquals(v * 3 * it, byteArray.length);
+    for (int i = 0; i < v * 3 * it; i++) {
+      assertEquals(i % (v * 3), byteArray[i]);
+    }
+    // verifying we have not created 500 * 23 / 10 slabs
+    assertTrue("slab count: " + capacityByteArrayOutputStream.getSlabCount(),capacityByteArrayOutputStream.getSlabCount() <= 20);
+    capacityByteArrayOutputStream.reset();
+    writeArraysOf3(capacityByteArrayOutputStream, v);
+    validate(capacityByteArrayOutputStream, v * 3);
+    // verifying we use less slabs now
+    assertTrue("slab count: " + capacityByteArrayOutputStream.getSlabCount(),capacityByteArrayOutputStream.getSlabCount() <= 2);
+  }
+
+  @Test
+  public void testReplaceByte() throws Throwable {
+    // test replace the first value
+    {
+      CapacityByteArrayOutputStream cbaos = new CapacityByteArrayOutputStream(5);
+      cbaos.write(10);
+      assertEquals(0, cbaos.getCurrentIndex());
+      cbaos.setByte(0, (byte) 7);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      cbaos.writeTo(baos);
+      assertEquals(7, baos.toByteArray()[0]);
+    }
+
+    // test replace value in the first slab
+    {
+      CapacityByteArrayOutputStream cbaos = new CapacityByteArrayOutputStream(5);
+      cbaos.write(10);
+      cbaos.write(13);
+      cbaos.write(15);
+      cbaos.write(17);
+      assertEquals(3, cbaos.getCurrentIndex());
+      cbaos.write(19);
+      cbaos.setByte(3, (byte) 7);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      cbaos.writeTo(baos);
+      assertArrayEquals(new byte[]{10, 13, 15, 7, 19}, baos.toByteArray());
+    }
+
+    // test replace in *not* the first slab
+    {
+      CapacityByteArrayOutputStream cbaos = new CapacityByteArrayOutputStream(5);
+
+      // advance part way through the 3rd slab
+      for (int i = 0; i < 12; i++) {
+        cbaos.write(100 + i);
+      }
+      assertEquals(11, cbaos.getCurrentIndex());
+
+      cbaos.setByte(6, (byte) 7);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      cbaos.writeTo(baos);
+      assertArrayEquals(
+        new byte[]{100, 101, 102, 103, 104, 105, 7, 107, 108, 109, 110, 111},
+        baos.toByteArray());
+    }
+
+    // test replace last value of a slab
+    {
+      CapacityByteArrayOutputStream cbaos = new CapacityByteArrayOutputStream(5);
+
+      // advance part way through the 3rd slab
+      for (int i = 0; i < 12; i++) {
+        cbaos.write(100 + i);
+      }
+      assertEquals(11, cbaos.getCurrentIndex());
+
+      cbaos.setByte(9, (byte) 7);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      cbaos.writeTo(baos);
+      assertArrayEquals(
+        new byte[]{100, 101, 102, 103, 104, 105, 106, 107, 108, 7, 110, 111},
+        baos.toByteArray());
+    }
+
+    // test replace last value
+    {
+      CapacityByteArrayOutputStream cbaos = new CapacityByteArrayOutputStream(5);
+
+      // advance part way through the 3rd slab
+      for (int i = 0; i < 12; i++) {
+        cbaos.write(100 + i);
+      }
+      assertEquals(11, cbaos.getCurrentIndex());
+
+      cbaos.setByte(11, (byte) 7);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      cbaos.writeTo(baos);
+      assertArrayEquals(
+        new byte[]{100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 7},
+        baos.toByteArray());
+    }
+
+  }
+
+  private void writeArraysOf3(CapacityByteArrayOutputStream capacityByteArrayOutputStream, int n)
+      throws IOException {
+    for (int i = 0; i < n; i++) {
+      byte[] toWrite = { (byte)(i * 3), (byte)(i * 3 + 1), (byte)(i * 3 + 2)};
+      capacityByteArrayOutputStream.write(toWrite);
+      assertEquals((i + 1) * 3, capacityByteArrayOutputStream.size());
+    }
+  }
+
+  private void validate(
+      CapacityByteArrayOutputStream capacityByteArrayOutputStream,
+      final int expectedSize) throws IOException {
+    final byte[] byteArray = BytesInput.from(capacityByteArrayOutputStream).toByteArray();
+    assertEquals(expectedSize, byteArray.length);
+    for (int i = 0; i < expectedSize; i++) {
+      assertEquals(i, byteArray[i]);
+    }
+  }
+}
diff --git a/parquet-column/src/test/java/parquet/column/primitive/TestBitPacking.java b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestBitPacking.java
similarity index 98%
rename from parquet-column/src/test/java/parquet/column/primitive/TestBitPacking.java
rename to parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestBitPacking.java
index 3cd3e63..d643093 100644
--- a/parquet-column/src/test/java/parquet/column/primitive/TestBitPacking.java
+++ b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestBitPacking.java
@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.primitive;
+package parquet.column.values.bitpacking;
 
 import static org.junit.Assert.assertArrayEquals;
 
@@ -21,11 +21,9 @@
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 
-
 import org.junit.Assert;
 import org.junit.Test;
 
-import parquet.column.values.bitpacking.BitPacking;
 import parquet.column.values.bitpacking.BitPacking.BitPackingReader;
 import parquet.column.values.bitpacking.BitPacking.BitPackingWriter;
 
diff --git a/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestByteBasedBitPackingEncoder.java b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestByteBasedBitPackingEncoder.java
new file mode 100644
index 0000000..9fd0e72
--- /dev/null
+++ b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestByteBasedBitPackingEncoder.java
@@ -0,0 +1,37 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+import org.junit.Test;
+
+public class TestByteBasedBitPackingEncoder {
+
+  @Test
+  public void testSlabBoundary() {
+    for (int i = 0; i < 32; i++) {
+      final ByteBasedBitPackingEncoder encoder = new ByteBasedBitPackingEncoder(i, Packer.BIG_ENDIAN);
+      // make sure to write more than a slab
+      for (int j = 0; j < 64 * 1024 * 32 + 10; j++) {
+        try {
+          encoder.writeInt(j);
+        } catch (Exception e) {
+          throw new RuntimeException(i + ": error writing " + j, e);
+        }
+      }
+    }
+  }
+
+}
diff --git a/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestByteBitPacking.java b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestByteBitPacking.java
new file mode 100644
index 0000000..72a227e
--- /dev/null
+++ b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestByteBitPacking.java
@@ -0,0 +1,146 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import parquet.column.values.bitpacking.BitPacking.BitPackingReader;
+import parquet.column.values.bitpacking.BitPacking.BitPackingWriter;
+
+public class TestByteBitPacking {
+
+  @Test
+  public void testPackUnPack() {
+    System.out.println();
+    System.out.println("testPackUnPack");
+    for (int i = 1; i < 32; i++) {
+      System.out.println("Width: " + i);
+      int[] unpacked = new int[32];
+      int[] values = generateValues(i);
+      packUnpack(Packer.BIG_ENDIAN.newBytePacker(i), values, unpacked);
+      System.out.println("Output: " + TestBitPacking.toString(unpacked));
+      Assert.assertArrayEquals("width "+i, values, unpacked);
+    }
+  }
+
+  private void packUnpack(BytePacker packer, int[] values, int[] unpacked) {
+    byte[] packed = new byte[packer.getBitWidth() * 4];
+    packer.pack32Values(values, 0, packed, 0);
+    System.out.println("packed: " + TestBitPacking.toString(packed));
+    packer.unpack32Values(packed, 0, unpacked, 0);
+  }
+
+  private int[] generateValues(int bitWidth) {
+    int[] values = new int[32];
+    for (int j = 0; j < values.length; j++) {
+      values[j] = (int)(Math.random() * 100000) % (int)Math.pow(2, bitWidth);
+    }
+    System.out.println("Input:  " + TestBitPacking.toString(values));
+    return values;
+  }
+
+  @Test
+  public void testPackUnPackAgainstHandWritten() throws IOException {
+    System.out.println();
+    System.out.println("testPackUnPackAgainstHandWritten");
+    for (int i = 1; i < 8; i++) {
+      System.out.println("Width: " + i);
+      byte[] packed = new byte[i * 4];
+      int[] unpacked = new int[32];
+      int[] values = generateValues(i);
+
+      // pack generated
+      final BytePacker packer = Packer.BIG_ENDIAN.newBytePacker(i);
+      packer.pack32Values(values, 0, packed, 0);
+
+      System.out.println("Generated: " + TestBitPacking.toString(packed));
+
+      // pack manual
+      final ByteArrayOutputStream manualOut = new ByteArrayOutputStream();
+      final BitPackingWriter writer = BitPacking.getBitPackingWriter(i, manualOut);
+      for (int j = 0; j < values.length; j++) {
+        writer.write(values[j]);
+      }
+      final byte[] packedManualAsBytes = manualOut.toByteArray();
+      System.out.println("Manual: " + TestBitPacking.toString(packedManualAsBytes));
+
+      // unpack manual
+      final BitPackingReader reader = BitPacking.createBitPackingReader(i, new ByteArrayInputStream(packed), 32);
+      for (int j = 0; j < unpacked.length; j++) {
+        unpacked[j] = reader.read();
+      }
+
+      System.out.println("Output: " + TestBitPacking.toString(unpacked));
+      Assert.assertArrayEquals("width " + i, values, unpacked);
+    }
+  }
+
+  @Test
+  public void testPackUnPackAgainstLemire() throws IOException {
+    for (Packer pack: Packer.values()) {
+      System.out.println();
+      System.out.println("testPackUnPackAgainstLemire " + pack.name());
+      for (int i = 1; i < 32; i++) {
+        System.out.println("Width: " + i);
+        int[] packed = new int[i];
+        int[] unpacked = new int[32];
+        int[] values = generateValues(i);
+
+        // pack lemire
+        final IntPacker packer = pack.newIntPacker(i);
+        packer.pack32Values(values, 0, packed, 0);
+        // convert to bytes
+        final ByteArrayOutputStream lemireOut = new ByteArrayOutputStream();
+        for (int v : packed) {
+          switch(pack) {
+          case LITTLE_ENDIAN:
+            lemireOut.write((v >>>  0) & 0xFF);
+            lemireOut.write((v >>>  8) & 0xFF);
+            lemireOut.write((v >>> 16) & 0xFF);
+            lemireOut.write((v >>> 24) & 0xFF);
+            break;
+          case BIG_ENDIAN:
+            lemireOut.write((v >>> 24) & 0xFF);
+            lemireOut.write((v >>> 16) & 0xFF);
+            lemireOut.write((v >>>  8) & 0xFF);
+            lemireOut.write((v >>>  0) & 0xFF);
+            break;
+          }
+        }
+        final byte[] packedByLemireAsBytes = lemireOut.toByteArray();
+        System.out.println("Lemire out: " + TestBitPacking.toString(packedByLemireAsBytes));
+
+        // pack manual
+        final BytePacker bytePacker = pack.newBytePacker(i);
+        byte[] packedGenerated = new byte[i * 4];
+        bytePacker.pack32Values(values, 0, packedGenerated, 0);
+        System.out.println("Gener. out: " + TestBitPacking.toString(packedGenerated));
+        Assert.assertEquals(pack.name() + " width " + i, TestBitPacking.toString(packedByLemireAsBytes), TestBitPacking.toString(packedGenerated));
+
+        bytePacker.unpack32Values(packedByLemireAsBytes, 0, unpacked, 0);
+        System.out.println("Output: " + TestBitPacking.toString(unpacked));
+
+        Assert.assertArrayEquals("width " + i, values, unpacked);
+      }
+    }
+  }
+}
diff --git a/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestLemireBitPacking.java b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestLemireBitPacking.java
new file mode 100644
index 0000000..8b41733
--- /dev/null
+++ b/parquet-encoding/src/test/java/parquet/column/values/bitpacking/TestLemireBitPacking.java
@@ -0,0 +1,118 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.column.values.bitpacking;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import parquet.column.values.bitpacking.BitPacking.BitPackingReader;
+import parquet.column.values.bitpacking.BitPacking.BitPackingWriter;
+
+public class TestLemireBitPacking {
+
+  @Test
+  public void testPackUnPack() {
+    for (Packer packer : Packer.values()) {
+      System.out.println();
+      System.out.println("testPackUnPack");
+      for (int i = 1; i < 32; i++) {
+        System.out.println("Width: " + i);
+        int[] values = generateValues(i);
+        int[] unpacked = new int[32];
+        {
+          packUnpack(packer.newIntPacker(i), values, unpacked);
+          System.out.println("int based Output " + packer.name() + ": " + TestBitPacking.toString(unpacked));
+          Assert.assertArrayEquals(packer.name() + " width "+i, values, unpacked);
+        }
+        {
+          packUnpack(packer.newBytePacker(i), values, unpacked);
+          System.out.println("byte based Output " + packer.name() + ": " + TestBitPacking.toString(unpacked));
+          Assert.assertArrayEquals(packer.name() + " width "+i, values, unpacked);
+        }
+      }
+    }
+  }
+
+  private void packUnpack(IntPacker packer, int[] values, int[] unpacked) {
+    int[] packed = new int[packer.getBitWidth()];
+    packer.pack32Values(values, 0, packed, 0);
+    packer.unpack32Values(packed, 0, unpacked, 0);
+  }
+
+  private void packUnpack(BytePacker packer, int[] values, int[] unpacked) {
+    byte[] packed = new byte[packer.getBitWidth() * 4];
+    packer.pack32Values(values, 0, packed, 0);
+    packer.unpack32Values(packed, 0, unpacked, 0);
+  }
+
+  private int[] generateValues(int bitWidth) {
+    int[] values = new int[32];
+    for (int j = 0; j < values.length; j++) {
+      values[j] = (int)(Math.random() * 100000) % (int)Math.pow(2, bitWidth);
+    }
+    System.out.println("Input:  " + TestBitPacking.toString(values));
+    return values;
+  }
+
+  @Test
+  public void testPackUnPackAgainstHandWritten() throws IOException {
+    System.out.println();
+    System.out.println("testPackUnPackAgainstHandWritten");
+    for (int i = 1; i < 8; i++) {
+      System.out.println("Width: " + i);
+      int[] packed = new int[i];
+      int[] unpacked = new int[32];
+      int[] values = generateValues(i);
+
+      // pack lemire
+      final IntPacker packer = Packer.BIG_ENDIAN.newIntPacker(i);
+      packer.pack32Values(values, 0, packed, 0);
+      // convert to ints
+      final ByteArrayOutputStream lemireOut = new ByteArrayOutputStream();
+      for (int v : packed) {
+        lemireOut.write((v >>> 24) & 0xFF);
+        lemireOut.write((v >>> 16) & 0xFF);
+        lemireOut.write((v >>>  8) & 0xFF);
+        lemireOut.write((v >>>  0) & 0xFF);
+      }
+      final byte[] packedByLemireAsBytes = lemireOut.toByteArray();
+      System.out.println("Lemire: " + TestBitPacking.toString(packedByLemireAsBytes));
+
+      // pack manual
+      final ByteArrayOutputStream manualOut = new ByteArrayOutputStream();
+      final BitPackingWriter writer = BitPacking.getBitPackingWriter(i, manualOut);
+      for (int j = 0; j < values.length; j++) {
+        writer.write(values[j]);
+      }
+      final byte[] packedManualAsBytes = manualOut.toByteArray();
+      System.out.println("Manual: " + TestBitPacking.toString(packedManualAsBytes));
+
+      // unpack manual
+      final BitPackingReader reader = BitPacking.createBitPackingReader(i, new ByteArrayInputStream(packedByLemireAsBytes), 32);
+      for (int j = 0; j < unpacked.length; j++) {
+        unpacked[j] = reader.read();
+      }
+
+      System.out.println("Output: " + TestBitPacking.toString(unpacked));
+      Assert.assertArrayEquals("width " + i, values, unpacked);
+    }
+  }
+
+}
diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml
new file mode 100644
index 0000000..343af01
--- /dev/null
+++ b/parquet-generator/pom.xml
@@ -0,0 +1,33 @@
+<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.0.0-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-generator</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Parquet Generator</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <properties>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+    </plugins>
+  </build>
+</project>
diff --git a/parquet-column/src/main/java/parquet/bytes/CapacityByteArrayOutputStream.java b/parquet-generator/src/main/java/parquet/encoding/Generator.java
similarity index 61%
rename from parquet-column/src/main/java/parquet/bytes/CapacityByteArrayOutputStream.java
rename to parquet-generator/src/main/java/parquet/encoding/Generator.java
index 0d73bb9..58d9382 100644
--- a/parquet-column/src/main/java/parquet/bytes/CapacityByteArrayOutputStream.java
+++ b/parquet-generator/src/main/java/parquet/encoding/Generator.java
@@ -13,28 +13,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.bytes;
+package parquet.encoding;
 
-import java.io.ByteArrayOutputStream;
+import parquet.encoding.bitpacking.ByteBasedBitPackingGenerator;
+import parquet.encoding.bitpacking.IntBasedBitPackingGenerator;
 
 /**
- * expose the memory used by a ByteArrayOutputStream
+ * main class for code generation hook in build
  *
  * @author Julien Le Dem
  *
  */
-public class CapacityByteArrayOutputStream extends ByteArrayOutputStream {
+public class Generator {
 
-  public CapacityByteArrayOutputStream(int initialSize) {
-    super(initialSize);
+  public static void main(String[] args) throws Exception {
+    IntBasedBitPackingGenerator.main(args);
+    ByteBasedBitPackingGenerator.main(args);
   }
 
-  /**
-   *
-   * @return the size of the allocated buffer
-   */
-  public int getCapacity() {
-    return buf.length;
-  }
-
-}
\ No newline at end of file
+}
diff --git a/parquet-generator/src/main/java/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java b/parquet-generator/src/main/java/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java
new file mode 100644
index 0000000..deb8f0e
--- /dev/null
+++ b/parquet-generator/src/main/java/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java
@@ -0,0 +1,256 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.encoding.bitpacking;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+import parquet.bytes.BytesUtils;
+
+/**
+ *
+ * This class generates bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ *
+ * TODO: remove the unnecessary masks for perf
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class ByteBasedBitPackingGenerator {
+
+  private static final String CLASS_NAME_PREFIX = "ByteBitPacking";
+  private static final int PACKER_COUNT = 32;
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    generateScheme(CLASS_NAME_PREFIX + "BE", true, basePath);
+    generateScheme(CLASS_NAME_PREFIX + "LE", false, basePath);
+  }
+
+  private static void generateScheme(String className, boolean msbFirst, String basePath) throws IOException {
+    final File file = new File(basePath + "/parquet/column/values/bitpacking/" + className + ".java").getAbsoluteFile();
+    if (!file.getParentFile().exists()) {
+      file.getParentFile().mkdirs();
+    }
+    FileWriter fw = new FileWriter(file);
+    fw.append("package parquet.column.values.bitpacking;\n");
+    fw.append("\n");
+    fw.append("/**\n");
+    if (msbFirst) {
+      fw.append(" * Packs from the Most Significant Bit first\n");
+    } else {
+      fw.append(" * Packs from the Least Significant Bit first\n");
+    }
+    fw.append(" * \n");
+    fw.append(" * @author automatically generated\n");
+    fw.append(" * @see ByteBasedBitPackingGenerator\n");
+    fw.append(" *\n");
+    fw.append(" */\n");
+    fw.append("public abstract class " + className + " {\n");
+    fw.append("\n");
+    fw.append("  private static final BytePacker[] packers = new BytePacker[33];\n");
+    fw.append("  static {\n");
+    for (int i = 0; i <= PACKER_COUNT; i++) {
+      fw.append("    packers[" + i + "] = new Packer" + i + "();\n");
+    }
+    fw.append("  }\n");
+    fw.append("\n");
+    fw.append("  public static final BytePackerFactory factory = new BytePackerFactory() {\n");
+    fw.append("    public BytePacker newBytePacker(int bitWidth) {\n");
+    fw.append("      return packers[bitWidth];\n");
+    fw.append("    }\n");
+    fw.append("  };\n");
+    fw.append("\n");
+    for (int i = 0; i <= PACKER_COUNT; i++) {
+      generateClass(fw, i, msbFirst);
+      fw.append("\n");
+    }
+    fw.append("}\n");
+    fw.close();
+  }
+
+  private static void generateClass(FileWriter fw, int bitWidth, boolean msbFirst) throws IOException {
+    fw.append("  private static final class Packer" + bitWidth + " extends BytePacker {\n");
+    fw.append("\n");
+    fw.append("    private Packer" + bitWidth + "() {\n");
+    fw.append("      super("+bitWidth+");\n");
+    fw.append("    }\n");
+    fw.append("\n");
+    // Packing
+    generatePack(fw, bitWidth, 1, msbFirst);
+    generatePack(fw, bitWidth, 4, msbFirst);
+
+    // Unpacking
+    generateUnpack(fw, bitWidth, 1, msbFirst);
+    generateUnpack(fw, bitWidth, 4, msbFirst);
+
+    fw.append("  }\n");
+  }
+
+  private static int getShift(FileWriter fw, int bitWidth, boolean msbFirst,
+      int byteIndex, int valueIndex) throws IOException {
+    // relative positions of the start and end of the value to the start and end of the byte
+    int valueStartBitIndex = (valueIndex * bitWidth) - (8 * (byteIndex));
+    int valueEndBitIndex = ((valueIndex + 1) * bitWidth) - (8 * (byteIndex + 1));
+
+    // boundaries of the current value that we want
+    int valueStartBitWanted;
+    int valueEndBitWanted;
+    // boundaries of the current byte that will receive them
+    int byteStartBitWanted;
+    int byteEndBitWanted;
+
+    int shift;
+
+    if (msbFirst) {
+      valueStartBitWanted = valueStartBitIndex < 0 ? bitWidth - 1 + valueStartBitIndex : bitWidth - 1;
+      valueEndBitWanted = valueEndBitIndex > 0 ? valueEndBitIndex : 0;
+      byteStartBitWanted = valueStartBitIndex < 0 ? 8 : 7 - valueStartBitIndex;
+      byteEndBitWanted = valueEndBitIndex > 0 ? 0 : -valueEndBitIndex;
+      shift = valueEndBitWanted - byteEndBitWanted;
+    } else {
+      valueStartBitWanted = bitWidth - 1 - (valueEndBitIndex > 0 ? valueEndBitIndex : 0);
+      valueEndBitWanted = bitWidth - 1 - (valueStartBitIndex < 0 ? bitWidth - 1 + valueStartBitIndex : bitWidth - 1);
+      byteStartBitWanted = 7 - (valueEndBitIndex > 0 ? 0 : -valueEndBitIndex);
+      byteEndBitWanted = 7 - (valueStartBitIndex < 0 ? 8 : 7 - valueStartBitIndex);
+      shift = valueStartBitWanted - byteStartBitWanted;
+    }
+
+    visualizeAlignment(
+        fw, bitWidth, valueEndBitIndex,
+        valueStartBitWanted, valueEndBitWanted,
+        byteStartBitWanted, byteEndBitWanted,
+        shift
+        );
+    return shift;
+  }
+
+  private static void visualizeAlignment(FileWriter fw, int bitWidth,
+      int valueEndBitIndex, int valueStartBitWanted, int valueEndBitWanted,
+      int byteStartBitWanted, int byteEndBitWanted, int shift) throws IOException {
+    // ASCII art to visualize what is happening
+    fw.append("//");
+    int buf = 2 + Math.max(0, bitWidth + 8);
+    for (int i = 0; i < buf; i++) {
+      fw.append(" ");
+    }
+    fw.append("[");
+    for (int i = 7; i >= 0; i--) {
+      if (i<=byteStartBitWanted && i>=byteEndBitWanted) {
+        fw.append(String.valueOf(i));
+      } else {
+        fw.append("_");
+      }
+    }
+    fw.append("]\n          //");
+    for (int i = 0; i < buf + (8 - bitWidth + shift); i++) {
+      fw.append(" ");
+    }
+    fw.append("[");
+    for (int i = bitWidth - 1; i >= 0 ; i--) {
+      if (i<=valueStartBitWanted && i>=valueEndBitWanted) {
+        fw.append(String.valueOf(i % 10));
+      } else {
+        fw.append("_");
+      }
+    }
+    fw.append("]\n");
+    fw.append("           ");
+  }
+
+  private static void generatePack(FileWriter fw, int bitWidth, int batch, boolean msbFirst) throws IOException {
+    int mask = genMask(bitWidth);
+    fw.append("    public final void pack" + (batch * 8) + "Values(final int[] in, final int inPos, final byte[] out, final int outPos) {\n");
+    for (int byteIndex = 0; byteIndex < bitWidth * batch; ++byteIndex) {
+      fw.append("      out[" + align(byteIndex, 2) + " + outPos] = (byte)((\n");
+      int startIndex = (byteIndex * 8) / bitWidth;
+      int endIndex = ((byteIndex + 1) * 8 + bitWidth - 1) / bitWidth;
+      for (int valueIndex = startIndex; valueIndex < endIndex; valueIndex++) {
+
+        if (valueIndex == startIndex) {
+          fw.append("          ");
+        } else {
+          fw.append("\n        | ");
+        }
+        int shift = getShift(fw, bitWidth, msbFirst, byteIndex, valueIndex);
+
+        String shiftString = ""; // used when shift == 0
+        if (shift > 0) {
+          shiftString = " >>> " + shift;
+        } else if (shift < 0) {
+          shiftString = " <<  " + ( - shift);
+        }
+        fw.append("((in[" + align(valueIndex, 2) + " + inPos] & " + mask + ")" + shiftString + ")");
+      }
+      fw.append(") & 255);\n");
+    }
+    fw.append("    }\n");
+  }
+
+  private static void generateUnpack(FileWriter fw, int bitWidth, int batch, boolean msbFirst)
+      throws IOException {
+    fw.append("    public final void unpack" + (batch * 8) + "Values(final byte[] in, final int inPos, final int[] out, final int outPos) {\n");
+    if (bitWidth > 0) {
+      int mask = genMask(bitWidth);
+      for (int valueIndex = 0; valueIndex < (batch * 8); ++valueIndex) {
+        fw.append("      out[" + align(valueIndex, 2) + " + outPos] =\n");
+
+        int startIndex = valueIndex * bitWidth / 8;
+        int endIndex = BytesUtils.paddedByteCountFromBits((valueIndex + 1) * bitWidth);
+
+        for (int byteIndex = startIndex; byteIndex < endIndex; byteIndex++) {
+          if (byteIndex == startIndex) {
+            fw.append("          ");
+          } else {
+            fw.append("\n        | ");
+          }
+          int shift = getShift(fw, bitWidth, msbFirst, byteIndex, valueIndex);
+
+          String shiftString = ""; // when shift == 0
+          if (shift < 0) {
+            shiftString = ">>>  " + (-shift);
+          } else if (shift > 0){
+            shiftString = "<<  " + shift;
+          }
+          fw.append(" (((((int)in[" + align(byteIndex, 2) + " + inPos]) & 255) " + shiftString + ") & " + mask + ")");
+        }
+        fw.append(";\n");
+      }
+    }
+    fw.append("    }\n");
+  }
+
+  private static int genMask(int bitWidth) {
+    int mask = 0;
+    for (int i = 0; i < bitWidth; i++) {
+      mask <<= 1;
+      mask |= 1;
+    }
+    return mask;
+  }
+
+  private static String align(int value, int digits) {
+    final String valueString = String.valueOf(value);
+    StringBuilder result = new StringBuilder();
+    for (int i = valueString.length(); i < digits; i++) {
+      result.append(" ");
+    }
+    result.append(valueString);
+    return result.toString();
+  }
+}
diff --git a/parquet-generator/src/main/java/parquet/encoding/bitpacking/IntBasedBitPackingGenerator.java b/parquet-generator/src/main/java/parquet/encoding/bitpacking/IntBasedBitPackingGenerator.java
new file mode 100644
index 0000000..8448c94
--- /dev/null
+++ b/parquet-generator/src/main/java/parquet/encoding/bitpacking/IntBasedBitPackingGenerator.java
@@ -0,0 +1,208 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.encoding.bitpacking;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+/**
+ * Scheme designed by D. Lemire
+ *
+ * This is a re-implementation of The scheme released under Apache License Version 2.0
+ * at https://github.com/lemire/JavaFastPFOR/blob/master/src/integercompression/BitPacking.java
+ *
+ * It generate two classes:
+ * - LemireBitPackingLE, the original scheme, filling the LSB first
+ * - LemireBitPackingBE, the scheme modified to fill the MSB first (and match our existing bit packing)
+ *
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ *
+ * The generated classes pack the values into arrays of ints (as opposed to arrays of bytes) based on a given bit width.
+ *
+ * Note: This is not really used for now as the hadoop API does not really let write int[]. We need to revisit this
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class IntBasedBitPackingGenerator {
+
+  private static final String CLASS_NAME_PREFIX = "LemireBitPacking";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    generateScheme(CLASS_NAME_PREFIX + "BE", true, basePath);
+    generateScheme(CLASS_NAME_PREFIX + "LE", false, basePath);
+  }
+
+  private static void generateScheme(String className, boolean msbFirst, String basePath) throws IOException {
+    final File file = new File(basePath + "/parquet/column/values/bitpacking/" + className + ".java").getAbsoluteFile();
+    if (!file.getParentFile().exists()) {
+      file.getParentFile().mkdirs();
+    }
+    FileWriter fw = new FileWriter(file);
+    fw.append("package parquet.column.values.bitpacking;\n");
+    fw.append("\n");
+    fw.append("/**\n");
+    fw.append(" * Based on the original implementation at at https://github.com/lemire/JavaFastPFOR/blob/master/src/integercompression/BitPacking.java\n");
+    fw.append(" * Which is released under the\n");
+    fw.append(" * Apache License Version 2.0 http://www.apache.org/licenses/.\n");
+    fw.append(" * By Daniel Lemire, http://lemire.me/en/\n");
+    fw.append(" * \n");
+    fw.append(" * Scheme designed by D. Lemire\n");
+    if (msbFirst) {
+      fw.append(" * Adapted to pack from the Most Significant Bit first\n");
+    }
+    fw.append(" * \n");
+    fw.append(" * @author automatically generated\n");
+    fw.append(" * @see IntBasedBitPackingGenerator\n");
+    fw.append(" *\n");
+    fw.append(" */\n");
+    fw.append("abstract class " + className + " {\n");
+    fw.append("\n");
+    fw.append("  private static final IntPacker[] packers = new IntPacker[32];\n");
+    fw.append("  static {\n");
+    for (int i = 0; i < 32; i++) {
+      fw.append("    packers[" + i + "] = new Packer" + i + "();\n");
+    }
+    fw.append("  }\n");
+    fw.append("\n");
+    fw.append("  public static final IntPackerFactory factory = new IntPackerFactory() {\n");
+    fw.append("    public IntPacker newIntPacker(int bitWidth) {\n");
+    fw.append("      return packers[bitWidth];\n");
+    fw.append("    }\n");
+    fw.append("  };\n");
+    fw.append("\n");
+    for (int i = 0; i < 32; i++) {
+      generateClass(fw, i, msbFirst);
+      fw.append("\n");
+    }
+    fw.append("}\n");
+    fw.close();
+  }
+
+  private static void generateClass(FileWriter fw, int bitWidth, boolean msbFirst) throws IOException {
+    int mask = 0;
+    for (int i = 0; i < bitWidth; i++) {
+      mask <<= 1;
+      mask |= 1;
+    }
+    fw.append("  private static final class Packer" + bitWidth + " extends IntPacker {\n");
+    fw.append("\n");
+    fw.append("    private Packer" + bitWidth + "() {\n");
+    fw.append("      super(" + bitWidth + ");\n");
+    fw.append("    }\n");
+    fw.append("\n");
+    // Packing
+    fw.append("    public final void pack32Values(final int[] in, final int inPos, final int[] out, final int outPos) {\n");
+    for (int i = 0; i < bitWidth; ++i) {
+      fw.append("      out[" + align(i, 2) + " + outPos] =\n");
+      int startIndex = (i * 32) / bitWidth;
+      int endIndex = ((i + 1) * 32 + bitWidth - 1) / bitWidth;
+      for (int j = startIndex; j < endIndex; j++) {
+        if (j == startIndex) {
+          fw.append("          ");
+        } else {
+          fw.append("\n        | ");
+        }
+        String shiftString = getPackShiftString(bitWidth, i, startIndex, j, msbFirst);
+        fw.append("((in[" + align(j, 2) + " + inPos] & " + mask + ")" + shiftString + ")");
+      }
+      fw.append(";\n");
+    }
+    fw.append("    }\n");
+
+    // Unpacking
+    fw.append("    public final void unpack32Values(final int[] in, final int inPos, final int[] out, final int outPos) {\n");
+    if (bitWidth > 0) {
+      for (int i = 0; i < 32; ++i) {
+        fw.append("      out[" + align(i, 2) + " + outPos] =");
+        int byteIndex = i * bitWidth / 32;
+        String shiftString = getUnpackShiftString(bitWidth, i, msbFirst);
+        fw.append(" ((in[" + align(byteIndex, 2) + " + inPos] " + shiftString + ") & " + mask + ")");
+        if (((i + 1) * bitWidth - 1 ) / 32 != byteIndex) {
+          // reading the end of the value from next int
+          int bitsRead = ((i + 1) * bitWidth - 1) % 32 + 1;
+          fw.append(" | ((in[" + align(byteIndex + 1, 2) + " + inPos]");
+          if (msbFirst) {
+            fw.append(") >>> " + align(32 - bitsRead, 2) + ")");
+          } else {
+            int lowerMask = 0;
+            for (int j = 0; j < bitsRead; j++) {
+              lowerMask <<= 1;
+              lowerMask |= 1;
+            }
+            fw.append(" & " + lowerMask + ") << " + align(bitWidth - bitsRead, 2) + ")");
+          }
+        }
+        fw.append(";\n");
+      }
+    }
+    fw.append("    }\n");
+    fw.append("  }\n");
+  }
+
+  private static String getUnpackShiftString(int bitWidth, int i, boolean msbFirst) {
+    final int regularShift = i * bitWidth % 32;
+    String shiftString;
+    if (msbFirst) {
+      int shift = 32 - (regularShift + bitWidth);
+      if (shift < 0) {
+        shiftString = "<<  " + align(-shift, 2);
+      } else {
+        shiftString = ">>> " + align(shift, 2);
+      }
+    } else {
+      shiftString = ">>> " + align(regularShift, 2);
+    }
+    return shiftString;
+  }
+
+  private static String getPackShiftString(int bitWidth, int integerIndex, int startIndex, int valueIndex, boolean msbFirst) {
+    String shiftString;
+    int regularShift = (valueIndex * bitWidth) % 32;
+    if (msbFirst) { // filling most significant bit first
+      int shift = 32 - (regularShift + bitWidth);
+      if (valueIndex == startIndex && (integerIndex * 32) % bitWidth != 0) {
+        // end of last value from previous int
+          shiftString = " <<  " + align(32 - (((valueIndex + 1) * bitWidth) % 32), 2);
+      } else if (shift < 0) {
+        // partial last value
+          shiftString = " >>> " + align(-shift, 2);
+      } else {
+        shiftString = " <<  " + align(shift, 2);
+      }
+    } else { // filling least significant bit first
+      if (valueIndex == startIndex && (integerIndex * 32) % bitWidth != 0) {
+        // end of last value from previous int
+        shiftString = " >>> " + align(32 - regularShift, 2);
+      } else {
+        shiftString = " <<  " + align(regularShift, 2);
+      }
+    }
+    return shiftString;
+  }
+
+  private static String align(int value, int digits) {
+    final String valueString = String.valueOf(value);
+    StringBuilder result = new StringBuilder();
+    for (int i = valueString.length(); i < digits; i++) {
+      result.append(" ");
+    }
+    result.append(valueString);
+    return result.toString();
+  }
+}
diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml
index 22dc08b..7be91e0 100644
--- a/parquet-hadoop/pom.xml
+++ b/parquet-hadoop/pom.xml
@@ -31,8 +31,8 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-core</artifactId>
-      <version>1.0.0</version>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
     </dependency>
     <dependency>
       <groupId>log4j</groupId>
@@ -40,11 +40,80 @@
       <version>1.2.17</version>
     </dependency>
     <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
       <version>11.0</version>
       <scope>test</scope>
     </dependency>
-
+    <dependency>
+      <groupId>org.xerial.snappy</groupId>
+      <artifactId>snappy-java</artifactId>
+      <version>1.0.5</version>
+      <type>jar</type>
+      <scope>compile</scope>
+    </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.0</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <artifactSet>
+                <includes>
+                  <include>org.codehaus.jackson:jackson-mapper-asl</include>
+                  <include>org.codehaus.jackson:jackson-core-asl</include>
+                </includes>
+              </artifactSet>
+              <relocations>
+                <relocation>
+                  <pattern>org.codehaus.jackson</pattern>
+                  <shadedPattern>parquet.org.codehaus.jackson</shadedPattern>
+                </relocation>
+              </relocations>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+
 </project>
diff --git a/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java
index 49f0228..76c75fb 100644
--- a/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java
+++ b/parquet-hadoop/src/main/java/parquet/format/converter/ParquetMetadataConverter.java
@@ -15,11 +15,15 @@
  */
 package parquet.format.converter;
 
+import static parquet.format.Util.readFileMetaData;
+import static parquet.format.Util.writePageHeader;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -27,13 +31,10 @@
 import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.thrift.TBase;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.transport.TIOStreamTransport;
-
+import parquet.Log;
 import parquet.format.ColumnChunk;
 import parquet.format.DataPageHeader;
+import parquet.format.DictionaryPageHeader;
 import parquet.format.Encoding;
 import parquet.format.FieldRepetitionType;
 import parquet.format.FileMetaData;
@@ -56,6 +57,7 @@
 import parquet.schema.TypeVisitor;
 
 public class ParquetMetadataConverter {
+  private static final Log LOG = Log.getLog(ParquetMetadataConverter.class);
 
   public FileMetaData toParquetMetadata(int currentVersion, ParquetMetadata parquetMetadata) {
     List<BlockMetaData> blocks = parquetMetadata.getBlocks();
@@ -77,6 +79,8 @@
       addKeyValue(fileMetaData, keyValue.getKey(), keyValue.getValue());
     }
 
+    fileMetaData.setCreated_by(parquetMetadata.getFileMetaData().getCreatedBy());
+
     return fileMetaData;
   }
 
@@ -137,6 +141,7 @@
           columnMetaData.getTotalSize(),
           columnMetaData.getFirstDataPageOffset()
           );
+      columnChunk.meta_data.dictionary_page_offset = columnMetaData.getDictionaryPageOffset();
 //      columnChunk.meta_data.index_page_offset = ;
 //      columnChunk.meta_data.key_value_metadata = ; // nothing yet
 
@@ -154,12 +159,55 @@
     return converted;
   }
 
+  private static final class EncodingList {
+
+    private final List<parquet.column.Encoding> encodings;
+
+    public EncodingList(List<parquet.column.Encoding> encodings) {
+      this.encodings = encodings;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof EncodingList) {
+        List<parquet.column.Encoding> other = ((EncodingList)obj).encodings;
+        if (other.size() != encodings.size()) {
+          return false;
+        }
+        for (int i = 0; i < other.size(); i++) {
+          if (!other.get(i).equals(encodings.get(i))) {
+            return false;
+          }
+        }
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = 1;
+      for (parquet.column.Encoding element : encodings)
+        result = 31 * result + (element == null ? 0 : element.hashCode());
+      return result;
+    }
+  }
+
+  private Map<EncodingList, List<parquet.column.Encoding>> encodingLists = new HashMap<EncodingList, List<parquet.column.Encoding>>();
+
   private List<parquet.column.Encoding> fromFormatEncodings(List<Encoding> encodings) {
     List<parquet.column.Encoding> converted = new ArrayList<parquet.column.Encoding>();
     for (Encoding encoding : encodings) {
       converted.add(getEncoding(encoding));
     }
-    return converted;
+    converted = Collections.unmodifiableList(converted);
+    EncodingList key = new EncodingList(converted);
+    List<parquet.column.Encoding> cached = encodingLists.get(key);
+    if (cached == null) {
+      cached = converted;
+      encodingLists.put(key, cached);
+    }
+    return cached;
   }
 
   public parquet.column.Encoding getEncoding(Encoding encoding) {
@@ -222,6 +270,14 @@
     fileMetaData.addToKey_value_metadata(keyValue);
   }
 
+  public ParquetMetadata readParquetMetadata(InputStream from) throws IOException {
+    FileMetaData fileMetaData = readFileMetaData(from);
+    if (Log.DEBUG) LOG.debug(fileMetaData);
+    ParquetMetadata parquetMetadata = fromParquetMetadata(fileMetaData);
+    if (Log.DEBUG) LOG.debug(ParquetMetadata.toPrettyJSON(parquetMetadata));
+    return parquetMetadata;
+  }
+
   public ParquetMetadata fromParquetMetadata(FileMetaData parquetMetadata) throws IOException {
     MessageType messageType = fromParquetSchema(parquetMetadata.getSchema());
     List<BlockMetaData> blocks = new ArrayList<BlockMetaData>();
@@ -238,13 +294,14 @@
           throw new ParquetDecodingException("all column chunks of the same row group must be in the same file for now");
         }
         parquet.format.ColumnMetaData metaData = columnChunk.meta_data;
-        String[] path = metaData.path_in_schema.toArray(new String[metaData.path_in_schema.size()]);
+        String[] path = getPath(metaData);
         ColumnChunkMetaData column = new ColumnChunkMetaData(
             path,
             messageType.getType(path).asPrimitiveType().getPrimitiveTypeName(),
             CompressionCodecName.fromParquet(metaData.codec),
             fromFormatEncodings(metaData.encodings));
         column.setFirstDataPageOffset(metaData.data_page_offset);
+        column.setDictionaryPageOffset(metaData.dictionary_page_offset);
         column.setValueCount(metaData.num_values);
         column.setTotalUncompressedSize(metaData.total_uncompressed_size);
         column.setTotalSize(metaData.total_compressed_size);
@@ -264,10 +321,49 @@
       }
     }
     return new ParquetMetadata(
-        new parquet.hadoop.metadata.FileMetaData(messageType, keyValueMetaData),
+        new parquet.hadoop.metadata.FileMetaData(messageType, keyValueMetaData, parquetMetadata.getCreated_by()),
         blocks);
   }
 
+
+  private static final class Path {
+
+    private final String[] p;
+
+    public Path(String[] path) {
+      this.p = path;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof Path) {
+        Arrays.equals(p, ((Path)obj).p);
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return Arrays.hashCode(p);
+    }
+  }
+
+  private Map<Path, String[]> paths = new HashMap<Path, String[]>();
+
+  private String[] getPath(parquet.format.ColumnMetaData metaData) {
+    String[] path = metaData.path_in_schema.toArray(new String[metaData.path_in_schema.size()]);
+    Path key = new Path(path);
+    String[] cached = paths.get(path);
+    if (cached == null) {
+      for (int i = 0; i < path.length; i++) {
+        path[i] = path[i].intern();
+      }
+      cached = path;
+      paths.put(key, cached);
+    }
+    return cached;
+  }
+
   MessageType fromParquetSchema(List<SchemaElement> schema) {
 
     Iterator<SchemaElement> iterator = schema.iterator();
@@ -319,68 +415,13 @@
     writePageHeader(newDataPageHeader(uncompressedSize, compressedSize, valueCount, rlEncoding, dlEncoding, valuesEncoding), to);
   }
 
-  protected void writePageHeader(PageHeader pageHeader, OutputStream to) throws IOException {
-    write(pageHeader, to);
-  }
-
-  public PageHeader readPageHeader(InputStream from) throws IOException {
-    return read(from, new PageHeader());
-  }
-
-  public void writeFileMetaData(parquet.format.FileMetaData fileMetadata, OutputStream to) throws IOException {
-    write(fileMetadata, to);
-  }
-
-  public parquet.format.FileMetaData readFileMetaData(InputStream from) throws IOException {
-    return read(from, new parquet.format.FileMetaData());
-  }
-
-  public String toString(TBase<?, ?> tbase) {
-    return tbase.toString();
-//    TMemoryBuffer trans = new TMemoryBuffer(1024);
-//    try {
-//      TSimpleJSONProtocol jsonProt = new TSimpleJSONProtocol(trans);
-//      tbase.write(jsonProt);
-//      return trans.toString("UTF-8");
-//    } catch (Exception e) { // TODO: cleanup exceptions
-//      throw new RuntimeException(e);
-//    }
-  }
-
-  private TCompactProtocol protocol(OutputStream to) {
-    return new TCompactProtocol(new TIOStreamTransport(to));
-  }
-
-  private TCompactProtocol protocol(InputStream from) {
-    return new TCompactProtocol(new TIOStreamTransport(from));
-  }
-
-  private <T extends TBase<?,?>> T read(InputStream from, T tbase)
-      throws IOException {
-    try {
-      tbase.read(protocol(from));
-      return tbase;
-    } catch (TException e) {
-      throw new IOException("can not read " + tbase.getClass() + ": " + e.getMessage(), e);
-    }
-  }
-
-  private void write(TBase<?, ?> tbase, OutputStream to)
-      throws IOException {
-    try {
-      tbase.write(protocol(to));
-    } catch (TException e) {
-      throw new IOException("can not write " + tbase, e);
-    }
-  }
-
   private PageHeader newDataPageHeader(
       int uncompressedSize, int compressedSize,
       int valueCount,
       parquet.column.Encoding rlEncoding,
       parquet.column.Encoding dlEncoding,
       parquet.column.Encoding valuesEncoding) {
-    PageHeader pageHeader = new PageHeader(PageType.DATA_PAGE, (int)uncompressedSize, (int)compressedSize);
+    PageHeader pageHeader = new PageHeader(PageType.DATA_PAGE, uncompressedSize, compressedSize);
     // TODO: pageHeader.crc = ...;
     pageHeader.data_page_header = new DataPageHeader(
         valueCount,
@@ -390,5 +431,13 @@
     return pageHeader;
   }
 
+  public void writeDictionaryPageHeader(
+      int uncompressedSize, int compressedSize, int valueCount,
+      parquet.column.Encoding valuesEncoding, OutputStream to) throws IOException {
+    PageHeader pageHeader = new PageHeader(PageType.DICTIONARY_PAGE, uncompressedSize, compressedSize);
+    pageHeader.dictionary_page_header = new DictionaryPageHeader(valueCount, getEncoding(valuesEncoding));
+    writePageHeader(pageHeader, to);
+  }
+
 
 }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageReadStore.java b/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageReadStore.java
index 78e6ae9..eb016d0 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageReadStore.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageReadStore.java
@@ -23,6 +23,7 @@
 
 import parquet.Log;
 import parquet.column.ColumnDescriptor;
+import parquet.column.page.DictionaryPage;
 import parquet.column.page.Page;
 import parquet.column.page.PageReadStore;
 import parquet.column.page.PageReader;
@@ -49,10 +50,12 @@
     private final BytesDecompressor decompressor;
     private final long valueCount;
     private final List<Page> compressedPages;
+    private final DictionaryPage compressedDictionaryPage;
 
-    ColumnChunkPageReader(BytesDecompressor decompressor, List<Page> compressedPages) {
+    ColumnChunkPageReader(BytesDecompressor decompressor, List<Page> compressedPages, DictionaryPage compressedDictionaryPage) {
       this.decompressor = decompressor;
       this.compressedPages = new LinkedList<Page>(compressedPages);
+      this.compressedDictionaryPage = compressedDictionaryPage;
       int count = 0;
       for (Page p : compressedPages) {
         count += p.getValueCount();
@@ -83,6 +86,21 @@
         throw new RuntimeException(e); // TODO: cleanup
       }
     }
+
+    @Override
+    public DictionaryPage readDictionaryPage() {
+      if (compressedDictionaryPage == null) {
+        return null;
+      }
+      try {
+        return new DictionaryPage(
+            decompressor.decompress(compressedDictionaryPage.getBytes(), compressedDictionaryPage.getUncompressedSize()),
+            compressedDictionaryPage.getDictionarySize(),
+            compressedDictionaryPage.getEncoding());
+      } catch (IOException e) {
+        throw new RuntimeException(e); // TODO: cleanup
+      }
+    }
   }
 
   private final Map<ColumnDescriptor, ColumnChunkPageReader> readers = new HashMap<ColumnDescriptor, ColumnChunkPageReader>();
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageWriteStore.java b/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageWriteStore.java
index b6d9dbb..48b7696 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageWriteStore.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ColumnChunkPageWriteStore.java
@@ -15,6 +15,8 @@
  */
 package parquet.hadoop;
 
+import static parquet.Log.INFO;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -23,17 +25,21 @@
 import java.util.Map;
 import java.util.Set;
 
+import parquet.Log;
 import parquet.bytes.BytesInput;
 import parquet.bytes.CapacityByteArrayOutputStream;
 import parquet.column.ColumnDescriptor;
 import parquet.column.Encoding;
+import parquet.column.page.DictionaryPage;
 import parquet.column.page.PageWriteStore;
 import parquet.column.page.PageWriter;
 import parquet.format.converter.ParquetMetadataConverter;
 import parquet.hadoop.CodecFactory.BytesCompressor;
+import parquet.io.ParquetEncodingException;
 import parquet.schema.MessageType;
 
 class ColumnChunkPageWriteStore implements PageWriteStore {
+  private static final Log LOG = Log.getLog(ColumnChunkPageWriteStore.class);
 
   private static ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
 
@@ -43,10 +49,12 @@
     private final BytesCompressor compressor;
 
     private final CapacityByteArrayOutputStream buf;
+    private DictionaryPage dictionaryPage;
 
     private long uncompressedLength;
     private long compressedLength;
     private long totalValueCount;
+    private int pageCount;
 
     private Set<Encoding> encodings = new HashSet<Encoding>();
 
@@ -72,6 +80,7 @@
       this.uncompressedLength += uncompressedSize;
       this.compressedLength += compressedSize;
       this.totalValueCount += valueCount;
+      this.pageCount += 1;
       compressedBytes.writeAllTo(buf);
       encodings.add(rlEncoding);
       encodings.add(dlEncoding);
@@ -85,30 +94,63 @@
 
     public void writeToFileWriter(ParquetFileWriter writer) throws IOException {
       writer.startColumn(path, totalValueCount, compressor.getCodecName());
+      if (dictionaryPage != null) {
+        writer.writeDictionaryPage(dictionaryPage);
+        encodings.add(dictionaryPage.getEncoding());
+      }
       writer.writeDataPages(BytesInput.from(buf), uncompressedLength, compressedLength, new ArrayList<Encoding>(encodings));
       writer.endColumn();
+      if (INFO) {
+        LOG.info(
+            String.format(
+                "written %,dB for %s: %,d values, %,dB raw, %,dB comp, %d pages, encodings: %s",
+                buf.size(), path, totalValueCount, uncompressedLength, compressedLength, pageCount, encodings)
+            + (dictionaryPage != null ? String.format(
+                    ", dic { %,d entries, %,dB raw, %,dB comp}",
+                    dictionaryPage.getDictionarySize(), dictionaryPage.getUncompressedSize(), dictionaryPage.getDictionarySize())
+                    : ""));
+      }
       encodings.clear();
+      pageCount = 0;
     }
 
     @Override
     public long allocatedSize() {
       return buf.getCapacity();
     }
+
+    @Override
+    public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException {
+      if (this.dictionaryPage != null) {
+        throw new ParquetEncodingException("Only one dictionary page is allowed");
+      }
+      BytesInput dictionaryBytes = dictionaryPage.getBytes();
+      int uncompressedSize = (int)dictionaryBytes.size();
+      BytesInput compressedBytes = compressor.compress(dictionaryBytes);
+      this.dictionaryPage = new DictionaryPage(BytesInput.copy(compressedBytes), uncompressedSize, dictionaryPage.getDictionarySize(), dictionaryPage.getEncoding());
+    }
+
+    @Override
+    public String memUsageString(String prefix) {
+      return buf.memUsageString(prefix + " ColumnChunkPageWriter");
+    }
   }
 
   private final Map<ColumnDescriptor, ColumnChunkPageWriter> writers = new HashMap<ColumnDescriptor, ColumnChunkPageWriter>();
   private final MessageType schema;
   private final BytesCompressor compressor;
+  private final int initialSize;
 
-  public ColumnChunkPageWriteStore(BytesCompressor compressor, MessageType schema) {
+  public ColumnChunkPageWriteStore(BytesCompressor compressor, MessageType schema, int initialSize) {
     this.compressor = compressor;
     this.schema = schema;
+    this.initialSize = initialSize;
   }
 
   @Override
   public PageWriter getPageWriter(ColumnDescriptor path) {
     if (!writers.containsKey(path)) {
-      writers.put(path,  new ColumnChunkPageWriter(path, compressor, 1024*1024/2)); // TODO: better deal with this initial size
+      writers.put(path,  new ColumnChunkPageWriter(path, compressor, initialSize));
     }
     return writers.get(path);
   }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
index 5903cb6..224fe56 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileReader.java
@@ -17,11 +17,12 @@
 
 import static parquet.Log.DEBUG;
 import static parquet.bytes.BytesUtils.readIntLittleEndian;
+import static parquet.format.Util.readPageHeader;
 import static parquet.hadoop.ParquetFileWriter.MAGIC;
 import static parquet.hadoop.ParquetFileWriter.PARQUET_METADATA_FILE;
 
+import java.io.Closeable;
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -46,16 +47,17 @@
 import parquet.Log;
 import parquet.bytes.BytesInput;
 import parquet.column.ColumnDescriptor;
+import parquet.column.page.DictionaryPage;
 import parquet.column.page.Page;
 import parquet.column.page.PageReadStore;
 import parquet.format.PageHeader;
-import parquet.format.PageType;
 import parquet.format.converter.ParquetMetadataConverter;
 import parquet.hadoop.CodecFactory.BytesDecompressor;
 import parquet.hadoop.ColumnChunkPageReadStore.ColumnChunkPageReader;
 import parquet.hadoop.metadata.BlockMetaData;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.io.ParquetDecodingException;
 
 /**
  * Reads a Parquet file
@@ -63,20 +65,12 @@
  * @author Julien Le Dem
  *
  */
-public class ParquetFileReader {
+public class ParquetFileReader implements Closeable {
 
   private static final Log LOG = Log.getLog(ParquetFileReader.class);
 
   private static ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
 
-  private static ParquetMetadata deserializeFooter(InputStream is) throws IOException {
-    parquet.format.FileMetaData parquetMetadata = parquetMetadataConverter.readFileMetaData(is);
-    if (Log.DEBUG) LOG.debug(parquetMetadataConverter.toString(parquetMetadata));
-    ParquetMetadata metadata = parquetMetadataConverter.fromParquetMetadata(parquetMetadata);
-    if (Log.DEBUG) LOG.debug(ParquetMetadata.toPrettyJSON(metadata));
-    return metadata;
-  }
-
   /**
    * for files provided, check if there's a summary file.
    * If a summary file is found it is used otherwise the file footer is used.
@@ -233,6 +227,12 @@
     return readFooter(configuration, fileSystem.getFileStatus(file));
   }
 
+
+  public static final List<Footer> readFooters(Configuration configuration, Path file) throws IOException {
+    FileSystem fileSystem = file.getFileSystem(configuration);
+    return readFooters(configuration, fileSystem.getFileStatus(file));
+  }
+
   /**
    * Reads the meta data block in the footer of the file
    * @param configuration
@@ -265,7 +265,7 @@
       throw new RuntimeException("corrupted file: the footer index is not within the file");
     }
     f.seek(footerIndex);
-    return deserializeFooter(f);
+    return parquetMetadataConverter.readParquetMetadata(f);
 
   }
   private CodecFactory codecFactory;
@@ -313,9 +313,14 @@
       String pathKey = Arrays.toString(mc.getPath());
       ColumnDescriptor columnDescriptor = paths.get(pathKey);
       if (columnDescriptor != null) {
-        List<Page> pagesInChunk = readColumnChunkPages(columnDescriptor, mc);
+        List<Page> pagesInChunk = new ArrayList<Page>();
+        List<DictionaryPage> dictionaryPagesInChunk = new ArrayList<DictionaryPage>();
+        readColumnChunkPages(columnDescriptor, mc, pagesInChunk, dictionaryPagesInChunk);
+        if (dictionaryPagesInChunk.size() > 1) {
+          throw new ParquetDecodingException("more than one dictionary page: " + dictionaryPagesInChunk);
+        }
         BytesDecompressor decompressor = codecFactory.getDecompressor(mc.getCodec());
-        ColumnChunkPageReader columnChunkPageReader = new ColumnChunkPageReader(decompressor, pagesInChunk);
+        ColumnChunkPageReader columnChunkPageReader = new ColumnChunkPageReader(decompressor, pagesInChunk, dictionaryPagesInChunk.size() == 0 ? null : dictionaryPagesInChunk.get(0));
         columnChunkPageReadStore.addColumn(columnDescriptor, columnChunkPageReader);
       }
     }
@@ -327,28 +332,48 @@
    * Read all of the pages in a given column chunk.
    * @return the list of pages
    */
-  private List<Page> readColumnChunkPages(ColumnDescriptor columnDescriptor, ColumnChunkMetaData metadata)
+  private void readColumnChunkPages(ColumnDescriptor columnDescriptor, ColumnChunkMetaData metadata, List<Page> pagesInChunk, List<DictionaryPage> dictionaryPagesInChunk)
       throws IOException {
-    f.seek(metadata.getFirstDataPageOffset());
+    long startingPos = metadata.getFirstDataPageOffset();
+    if (metadata.getDictionaryPageOffset() > 0 && metadata.getDictionaryPageOffset() < startingPos) {
+      // if there's a dictionary and it's before the first data page, start from there
+      startingPos = metadata.getDictionaryPageOffset();
+    }
+    f.seek(startingPos);
     if (DEBUG) {
       LOG.debug(f.getPos() + ": start column chunk " + Arrays.toString(metadata.getPath()) +
         " " + metadata.getType() + " count=" + metadata.getValueCount());
     }
-
-    List<Page> pagesInChunk = new ArrayList<Page>();
     long valuesCountReadSoFar = 0;
     while (valuesCountReadSoFar < metadata.getValueCount()) {
-      PageHeader pageHeader = readNextDataPageHeader();
-      pagesInChunk.add(
-          new Page(
-              BytesInput.copy(BytesInput.from(f, pageHeader.compressed_page_size)),
-              pageHeader.data_page_header.num_values,
-              pageHeader.uncompressed_page_size,
-              parquetMetadataConverter.getEncoding(pageHeader.data_page_header.repetition_level_encoding),
-              parquetMetadataConverter.getEncoding(pageHeader.data_page_header.definition_level_encoding),
-              parquetMetadataConverter.getEncoding(pageHeader.data_page_header.encoding)
-              ));
-      valuesCountReadSoFar += pageHeader.data_page_header.num_values;
+      PageHeader pageHeader = readPageHeader(f);
+      switch (pageHeader.type) {
+        case DICTIONARY_PAGE:
+          dictionaryPagesInChunk.add(
+              new DictionaryPage(
+                  BytesInput.copy(BytesInput.from(f, pageHeader.compressed_page_size)),
+                  pageHeader.uncompressed_page_size,
+                  pageHeader.dictionary_page_header.num_values,
+                  parquetMetadataConverter.getEncoding(pageHeader.dictionary_page_header.encoding)
+                  ));
+          break;
+        case DATA_PAGE:
+          pagesInChunk.add(
+              new Page(
+                  BytesInput.copy(BytesInput.from(f, pageHeader.compressed_page_size)),
+                  pageHeader.data_page_header.num_values,
+                  pageHeader.uncompressed_page_size,
+                  parquetMetadataConverter.getEncoding(pageHeader.data_page_header.repetition_level_encoding),
+                  parquetMetadataConverter.getEncoding(pageHeader.data_page_header.definition_level_encoding),
+                  parquetMetadataConverter.getEncoding(pageHeader.data_page_header.encoding)
+                  ));
+          valuesCountReadSoFar += pageHeader.data_page_header.num_values;
+          break;
+        default:
+          if (DEBUG) LOG.debug("skipping page of type " + pageHeader.type + " of size " + pageHeader.compressed_page_size);
+          f.skip(pageHeader.compressed_page_size);
+          break;
+      }
     }
     if (valuesCountReadSoFar != metadata.getValueCount()) {
       // Would be nice to have a CorruptParquetFileException or something as a subclass?
@@ -358,27 +383,9 @@
           " but got " + valuesCountReadSoFar + " values instead over " + pagesInChunk.size()
           + " pages ending at file offset " + f.getPos());
     }
-    return pagesInChunk;
   }
 
-  private PageHeader readNextDataPageHeader() throws IOException {
-    PageHeader pageHeader;
-    do {
-      long pos = f.getPos();
-      if (DEBUG) LOG.debug(pos + ": reading page");
-      try {
-        pageHeader = parquetMetadataConverter.readPageHeader(f);
-        if (pageHeader.type != PageType.DATA_PAGE) {
-          if (DEBUG) LOG.debug("not a data page, skipping " + pageHeader.compressed_page_size);
-          f.skip(pageHeader.compressed_page_size);
-        }
-      } catch (IOException e) {
-        throw new IOException("could not read page header at position " + pos, e);
-      }
-    } while (pageHeader.type != PageType.DATA_PAGE);
-    return pageHeader;
-  }
-
+  @Override
   public void close() throws IOException {
     f.close();
     this.codecFactory.release();
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
index 6165a37..49ec9b1 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
@@ -17,6 +17,7 @@
 
 import static parquet.Log.DEBUG;
 import static parquet.Log.INFO;
+import static parquet.format.Util.writeFileMetaData;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
@@ -34,9 +35,11 @@
 import org.apache.hadoop.fs.Path;
 
 import parquet.Log;
+import parquet.Version;
 import parquet.bytes.BytesInput;
 import parquet.bytes.BytesUtils;
 import parquet.column.ColumnDescriptor;
+import parquet.column.page.DictionaryPage;
 import parquet.format.converter.ParquetMetadataConverter;
 import parquet.hadoop.metadata.BlockMetaData;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
@@ -175,10 +178,36 @@
   }
 
   /**
+   * writes a dictionary page page
+   * @param dictionaryPage the dictionary page
+   */
+  public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException {
+    state = state.write();
+    if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page: " + dictionaryPage.getDictionarySize() + " values");
+    currentColumn.setDictionaryPageOffset(out.getPos());
+    int uncompressedSize = dictionaryPage.getUncompressedSize();
+    int compressedPageSize = (int)dictionaryPage.getBytes().size(); // TODO: fix casts
+    metadataConverter.writeDictionaryPageHeader(
+        uncompressedSize,
+        compressedPageSize,
+        dictionaryPage.getDictionarySize(),
+        dictionaryPage.getEncoding(),
+        out);
+    this.uncompressedLength += uncompressedSize;
+    this.compressedLength += compressedPageSize;
+    if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page content " + compressedPageSize);
+    dictionaryPage.getBytes().writeAllTo(out);
+    currentEncodings.add(dictionaryPage.getEncoding());
+  }
+
+  /**
    * writes a single page
    * @param valueCount count of values
    * @param uncompressedPageSize the size of the data once uncompressed
    * @param bytes the compressed data for the page without header
+   * @param rlEncoding encoding of the repetition level
+   * @param dlEncoding encoding of the definition level
+   * @param valuesEncoding encoding of values
    */
   public void writeDataPage(
       int valueCount, int uncompressedPageSize,
@@ -187,7 +216,8 @@
       parquet.column.Encoding dlEncoding,
       parquet.column.Encoding valuesEncoding) throws IOException {
     state = state.write();
-    if (DEBUG) LOG.debug(out.getPos() + ": write data page: " + valueCount + " values");
+    long beforeHeader = out.getPos();
+    if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values");
     int compressedPageSize = (int)bytes.size();
     metadataConverter.writeDataPageHeader(
         uncompressedPageSize, compressedPageSize,
@@ -196,8 +226,9 @@
         dlEncoding,
         valuesEncoding,
         out);
-    this.uncompressedLength += uncompressedPageSize;
-    this.compressedLength += compressedPageSize;
+    long headerSize = out.getPos() - beforeHeader;
+    this.uncompressedLength += uncompressedPageSize + headerSize;
+    this.compressedLength += compressedPageSize + headerSize;
     if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize);
     bytes.writeAllTo(out);
     currentEncodings.add(rlEncoding);
@@ -215,8 +246,9 @@
    void writeDataPages(BytesInput bytes, long uncompressedTotalPageSize, long compressedTotalPageSize, List<parquet.column.Encoding> encodings) throws IOException {
     state = state.write();
     if (DEBUG) LOG.debug(out.getPos() + ": write data pages");
-    this.uncompressedLength += uncompressedTotalPageSize;
-    this.compressedLength += compressedTotalPageSize;
+    long headersSize = bytes.size() - compressedTotalPageSize;
+    this.uncompressedLength += uncompressedTotalPageSize + headersSize;
+    this.compressedLength += compressedTotalPageSize + headersSize;
     if (DEBUG) LOG.debug(out.getPos() + ": write data pages content");
     bytes.writeAllTo(out);
     currentEncodings.addAll(encodings);
@@ -233,7 +265,7 @@
     currentColumn.setTotalSize(compressedLength);
     currentColumn.getEncodings().addAll(currentEncodings);
     currentBlock.addColumn(currentColumn);
-    if (INFO) LOG.info("ended Column chumk: " + currentColumn);
+    if (DEBUG) LOG.info("ended Column chumk: " + currentColumn);
     currentColumn = null;
     this.uncompressedLength = 0;
     this.compressedLength = 0;
@@ -261,7 +293,7 @@
   public void end(Map<String, String> extraMetaData) throws IOException {
     state = state.end();
     if (DEBUG) LOG.debug(out.getPos() + ": end");
-    ParquetMetadata footer = new ParquetMetadata(new FileMetaData(schema, extraMetaData), blocks);
+    ParquetMetadata footer = new ParquetMetadata(new FileMetaData(schema, extraMetaData, Version.FULL_VERSION), blocks);
     serializeFooter(footer, out);
     out.close();
   }
@@ -269,7 +301,7 @@
   private static void serializeFooter(ParquetMetadata footer, FSDataOutputStream out) throws IOException {
     long footerIndex = out.getPos();
     parquet.format.FileMetaData parquetMetadata = new ParquetMetadataConverter().toParquetMetadata(CURRENT_VERSION, footer);
-    metadataConverter.writeFileMetaData(parquetMetadata, out);
+    writeFileMetaData(parquetMetadata, out);
     if (DEBUG) LOG.debug(out.getPos() + ": footer length = " + (out.getPos() - footerIndex));
     BytesUtils.writeIntLittleEndian(out, (int)(out.getPos() - footerIndex));
     out.write(MAGIC);
@@ -308,13 +340,22 @@
     return new ParquetMetadata(fileMetaData, blocks);
   }
 
+  /**
+   * @return the current position in the underlying file
+   * @throws IOException
+   */
+  public long getPos() throws IOException {
+    return out.getPos();
+  }
+
   static FileMetaData mergeInto(
       FileMetaData toMerge,
       FileMetaData mergedMetadata) {
     if (mergedMetadata == null) {
       return new FileMetaData(
           toMerge.getSchema(),
-          new HashMap<String, String>(toMerge.getKeyValueMetaData()));
+          new HashMap<String, String>(toMerge.getKeyValueMetaData()),
+          Version.FULL_VERSION);
     } else if (
         (mergedMetadata.getSchema() == null && toMerge.getSchema() != null)
         || (mergedMetadata.getSchema() != null && !mergedMetadata.getSchema().equals(toMerge.getSchema()))) {
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
index 53f7ebf..b6cb762 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
@@ -22,12 +22,14 @@
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -39,8 +41,10 @@
 import parquet.hadoop.api.ReadSupport;
 import parquet.hadoop.api.ReadSupport.ReadContext;
 import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.metadata.ColumnChunkMetaData;
 import parquet.hadoop.metadata.FileMetaData;
 import parquet.hadoop.metadata.ParquetMetadata;
+import parquet.hadoop.util.ContextUtil;
 import parquet.schema.MessageType;
 import parquet.schema.MessageTypeParser;
 
@@ -63,7 +67,11 @@
   public static final String READ_SUPPORT_CLASS = "parquet.read.support.class";
 
   public static void setReadSupportClass(Job job,  Class<?> readSupportClass) {
-    job.getConfiguration().set(READ_SUPPORT_CLASS, readSupportClass.getName());
+    ContextUtil.getConfiguration(job).set(READ_SUPPORT_CLASS, readSupportClass.getName());
+  }
+
+  public static void setReadSupportClass(JobConf conf, Class<?> readSupportClass) {
+    conf.set(READ_SUPPORT_CLASS, readSupportClass.getName());
   }
 
   public static Class<?> getReadSupportClass(Configuration configuration) {
@@ -87,7 +95,7 @@
   private List<Footer> footers;
 
   /**
-   * Hadoop will instanciate using this constructor
+   * Hadoop will instantiate using this constructor
    */
   public ParquetInputFormat() {
   }
@@ -107,10 +115,10 @@
   public RecordReader<Void, T> createRecordReader(
       InputSplit inputSplit,
       TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
-    return new ParquetRecordReader<T>(getReadSupport(taskAttemptContext.getConfiguration()));
+    return new ParquetRecordReader<T>(getReadSupport(ContextUtil.getConfiguration(taskAttemptContext)));
   }
 
-  private ReadSupport<T> getReadSupport(Configuration configuration){
+  public ReadSupport<T> getReadSupport(Configuration configuration){
     try {
       if (readSupportClass == null) {
         readSupportClass = getReadSupportClass(configuration);
@@ -125,30 +133,26 @@
     }
   }
 
-  private String getRequestedSchema(String fileSchema, String requestedSchema) {
-    if (requestedSchema != null) {
-      MessageType requestedMessageType = MessageTypeParser.parseMessageType(requestedSchema);
-      MessageType fileMessageType = MessageTypeParser.parseMessageType(fileSchema);
-      fileMessageType.checkContains(requestedMessageType);
-      return requestedSchema;
-    }
-    return fileSchema;
-  }
-
   /**
    * groups together all the data blocks for the same HDFS block
    * @param blocks data blocks (row groups)
    * @param hdfsBlocks hdfs blocks
    * @param fileStatus the containing file
    * @param fileMetaData file level meta data
-   * @param extraMetadata
-   * @param readSupport how to materialize the records
+   * @param readSupportClass the class used to materialize records
+   * @param requestedSchema the schema requested by the user
+   * @param readSupportMetadata the metadata provided by the readSupport implementation in init
    * @return the splits (one per HDFS block)
    * @throws IOException If hosts can't be retrieved for the HDFS block
    */
-  static <T> List<InputSplit> generateSplits(List<BlockMetaData> blocks,
-      BlockLocation[] hdfsBlocks, FileStatus fileStatus,
-      FileMetaData fileMetaData, Class<?> readSupportClass, String requestedSchema) throws IOException {
+  static <T> List<ParquetInputSplit> generateSplits(
+      List<BlockMetaData> blocks,
+      BlockLocation[] hdfsBlocks,
+      FileStatus fileStatus,
+      FileMetaData fileMetaData,
+      Class<?> readSupportClass,
+      String requestedSchema,
+      Map<String, String> readSupportMetadata) throws IOException {
     Comparator<BlockLocation> comparator = new Comparator<BlockLocation>() {
       @Override
       public int compare(BlockLocation b1, BlockLocation b2) {
@@ -179,22 +183,34 @@
         }
       }
     }
-    List<InputSplit> splits = new ArrayList<InputSplit>();
+    List<ParquetInputSplit> splits = new ArrayList<ParquetInputSplit>();
     for (int i = 0; i < hdfsBlocks.length; i++) {
       BlockLocation hdfsBlock = hdfsBlocks[i];
       List<BlockMetaData> blocksForCurrentSplit = splitGroups.get(i);
       if (blocksForCurrentSplit.size() == 0) {
         LOG.warn("HDFS block without row group: " + hdfsBlocks[i]);
       } else {
+        long length = 0;
+        for (BlockMetaData block : blocksForCurrentSplit) {
+          MessageType requested = MessageTypeParser.parseMessageType(requestedSchema);
+          List<ColumnChunkMetaData> columns = block.getColumns();
+          for (ColumnChunkMetaData column : columns) {
+            if (requested.containsPath(column.getPath())) {
+              length += column.getTotalSize();
+            }
+          }
+        }
         splits.add(new ParquetInputSplit(
           fileStatus.getPath(),
           hdfsBlock.getOffset(),
-          hdfsBlock.getLength(),
+          length,
           hdfsBlock.getHosts(),
           blocksForCurrentSplit,
           fileMetaData.getSchema().toString(),
           requestedSchema,
-          fileMetaData.getKeyValueMetaData()
+          fileMetaData.getSchema().toString(),
+          fileMetaData.getKeyValueMetaData(),
+          readSupportMetadata
           ));
       }
     }
@@ -207,9 +223,13 @@
   @Override
   public List<InputSplit> getSplits(JobContext jobContext) throws IOException {
     List<InputSplit> splits = new ArrayList<InputSplit>();
-    Configuration configuration = jobContext.getConfiguration();
-    List<Footer> footers = getFooters(jobContext);
-    FileMetaData globalMetaData = getGlobalMetaData(jobContext);
+    splits.addAll(getSplits(ContextUtil.getConfiguration(jobContext), getFooters(jobContext)));
+    return splits;
+  }
+
+  public List<ParquetInputSplit> getSplits(Configuration configuration, List<Footer> footers) throws IOException {
+    List<ParquetInputSplit> splits = new ArrayList<ParquetInputSplit>();
+    FileMetaData globalMetaData = getGlobalMetaData(footers);
     ReadContext readContext = getReadSupport(configuration).init(
         configuration,
         globalMetaData.getKeyValueMetaData(),
@@ -229,7 +249,8 @@
               fileStatus,
               parquetMetaData.getFileMetaData(),
               readSupportClass,
-              readContext.getRequestedSchema().toString())
+              readContext.getRequestedSchema().toString(),
+              readContext.getReadSupportMetadata())
           );
     }
     return splits;
@@ -242,26 +263,32 @@
    */
   public List<Footer> getFooters(JobContext jobContext) throws IOException {
     if (footers == null) {
-      Configuration configuration = jobContext.getConfiguration();
-      List<FileStatus> statuses = super.listStatus(jobContext);
-      LOG.debug("reading " + statuses.size() + " files");
-      footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(configuration, statuses);
+      footers = getFooters(ContextUtil.getConfiguration(jobContext), super.listStatus(jobContext));
     }
+
     return footers;
   }
 
+  public List<Footer> getFooters(Configuration configuration, List<FileStatus> statuses) throws IOException {
+      LOG.debug("reading " + statuses.size() + " files");
+      return ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(configuration, statuses);
+  }
+
   /**
    * @param jobContext the current job context
    * @return the merged metadata from the footers
    * @throws IOException
    */
   public FileMetaData getGlobalMetaData(JobContext jobContext) throws IOException {
+    return getGlobalMetaData(getFooters(jobContext));
+  }
+
+  private FileMetaData getGlobalMetaData(List<Footer> footers) throws IOException {
     FileMetaData fileMetaData = null;
-    for (Footer footer : getFooters(jobContext)) {
+    for (Footer footer : footers) {
       ParquetMetadata currentMetadata = footer.getParquetMetadata();
       fileMetaData = mergeInto(currentMetadata.getFileMetaData(), fileMetaData);
     }
     return fileMetaData;
   }
-
 }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java
index 23fe18b..6a228d8 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputSplit.java
@@ -51,7 +51,9 @@
   private List<BlockMetaData> blocks;
   private String schema;
   private String requestedSchema;
+  private String fileSchema;
   private Map<String, String> extraMetadata;
+  private Map<String, String> readSupportMetadata;
 
 
   /**
@@ -69,8 +71,10 @@
    * @param blocks the block meta data (Columns locations)
    * @param schema the file schema
    * @param readSupportClass the class used to materialize records
-   * @param requestedSchema the requested schema for projection
+   * @param requestedSchema the requested schema for materialization
+   * @param fileSchema the schema of the file
    * @param extraMetadata the app specific meta data in the file
+   * @param readSupportMetadata the read support specific metadata
    */
   public ParquetInputSplit(
       Path path,
@@ -80,7 +84,9 @@
       List<BlockMetaData> blocks,
       String schema,
       String requestedSchema,
-      Map<String, String> extraMetadata) {
+      String fileSchema,
+      Map<String, String> extraMetadata,
+      Map<String, String> readSupportMetadata) {
     this.path = path.toUri().toString();
     this.start = start;
     this.length = length;
@@ -88,7 +94,9 @@
     this.blocks = blocks;
     this.schema = schema;
     this.requestedSchema = requestedSchema;
+    this.fileSchema = fileSchema;
     this.extraMetadata = extraMetadata;
+    this.readSupportMetadata = readSupportMetadata;
   }
 
   /**
@@ -147,12 +155,25 @@
   }
 
   /**
-   * @return app specific metadata
+   * @return the file schema
+   */
+  public String getFileSchema() {
+    return fileSchema;
+  }
+
+  /**
+   * @return app specific metadata from the file
    */
   public Map<String, String> getExtraMetadata() {
     return extraMetadata;
   }
 
+  /**
+   * @return app specific metadata provided by the read support in the init phase
+   */
+  public Map<String, String> getReadSupportMetadata() {
+    return readSupportMetadata;
+  }
 
   /**
    * {@inheritDoc}
@@ -171,7 +192,9 @@
       this.blocks = other.blocks;
       this.schema = other.schema;
       this.requestedSchema = other.requestedSchema;
+      this.fileSchema = other.fileSchema;
       this.extraMetadata = other.extraMetadata;
+      this.readSupportMetadata = other.readSupportMetadata;
     } catch (ClassNotFoundException e) {
       throw new IOException("wrong class serialized", e);
     }
@@ -199,7 +222,10 @@
         + " blocks: " + blocks.size()
         + " schema: " + schema
         + " requestedSchema: " + requestedSchema
+        + " fileSchema: " + fileSchema
         + " extraMetadata: " + extraMetadata
+        + " readSupportMetadata: " + readSupportMetadata
         + "}";
   }
+
 }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputCommitter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputCommitter.java
index b923552..31917d2 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputCommitter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputCommitter.java
@@ -28,6 +28,7 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 
 import parquet.Log;
+import parquet.hadoop.util.ContextUtil;
 
 public class ParquetOutputCommitter extends FileOutputCommitter {
   private static final Log LOG = Log.getLog(ParquetOutputCommitter.class);
@@ -42,7 +43,7 @@
   public void commitJob(JobContext jobContext) throws IOException {
     super.commitJob(jobContext);
     try {
-      Configuration configuration = jobContext.getConfiguration();
+      Configuration configuration = ContextUtil.getConfiguration(jobContext);
       final FileSystem fileSystem = outputPath.getFileSystem(configuration);
       FileStatus outputStatus = fileSystem.getFileStatus(outputPath);
       List<Footer> footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus);
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java
index 7208811..99d2eba 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetOutputFormat.java
@@ -16,6 +16,11 @@
 package parquet.hadoop;
 
 import static parquet.Log.INFO;
+import static parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
+import static parquet.hadoop.ParquetWriter.DEFAULT_PAGE_SIZE;
+import static parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
+import static parquet.hadoop.metadata.CompressionCodecName.fromConf;
+import static parquet.hadoop.util.ContextUtil.getConfiguration;
 
 import java.io.IOException;
 
@@ -59,6 +64,9 @@
  * # The write support class to convert the records written to the OutputFormat into the events accepted by the record consumer
  * # Usually provided by a specific ParquetOutputFormat subclass
  * parquet.write.support.class= # fully qualified name
+ *
+ * # To enable dictionary encoding
+ * parquet.enable.dictionary=false # true to enable dictionary encoding
  * </pre>
  *
  * If parquet.compression is not set, the following properties are checked (FileOutputFormat behavior).
@@ -81,13 +89,15 @@
   public static final String PAGE_SIZE           = "parquet.page.size";
   public static final String COMPRESSION         = "parquet.compression";
   public static final String WRITE_SUPPORT_CLASS = "parquet.write.support.class";
+  public static final String ENABLE_DICTIONARY   = "parquet.enable.dictionary";
+  public static final String VALIDATION          = "parquet.validation";
 
   public static void setWriteSupportClass(Job job,  Class<?> writeSupportClass) {
-    job.getConfiguration().set(WRITE_SUPPORT_CLASS, writeSupportClass.getName());
+    getConfiguration(job).set(WRITE_SUPPORT_CLASS, writeSupportClass.getName());
   }
 
   public static Class<?> getWriteSupportClass(JobContext jobContext) {
-    final String className = jobContext.getConfiguration().get(WRITE_SUPPORT_CLASS);
+    final String className = getConfiguration(jobContext).get(WRITE_SUPPORT_CLASS);
     if (className == null) {
       return null;
     }
@@ -103,31 +113,47 @@
   }
 
   public static void setBlockSize(Job job, int blockSize) {
-    job.getConfiguration().setInt(BLOCK_SIZE, blockSize);
+    getConfiguration(job).setInt(BLOCK_SIZE, blockSize);
   }
 
   public static void setPageSize(Job job, int pageSize) {
-    job.getConfiguration().setInt(PAGE_SIZE, pageSize);
+    getConfiguration(job).setInt(PAGE_SIZE, pageSize);
   }
 
   public static void setCompression(Job job, CompressionCodecName compression) {
-    job.getConfiguration().set(COMPRESSION, compression.name());
+    getConfiguration(job).set(COMPRESSION, compression.name());
+  }
+
+  public static void setEnableDictionary(Job job, boolean enableDictionary) {
+    getConfiguration(job).setBoolean(ENABLE_DICTIONARY, enableDictionary);
+  }
+
+  public static boolean getEnableDictionary(JobContext jobContext) {
+    return getConfiguration(jobContext).getBoolean(ENABLE_DICTIONARY, false);
   }
 
   public static int getBlockSize(JobContext jobContext) {
-    return jobContext.getConfiguration().getInt(BLOCK_SIZE, 50*1024*1024);
+    return getConfiguration(jobContext).getInt(BLOCK_SIZE, DEFAULT_BLOCK_SIZE);
   }
 
   public static int getPageSize(JobContext jobContext) {
-    return jobContext.getConfiguration().getInt(PAGE_SIZE, 1*1024*1024);
+    return getConfiguration(jobContext).getInt(PAGE_SIZE, DEFAULT_PAGE_SIZE);
   }
 
   public static CompressionCodecName getCompression(JobContext jobContext) {
-    return CompressionCodecName.fromConf(jobContext.getConfiguration().get(COMPRESSION, CompressionCodecName.UNCOMPRESSED.name()));
+    return fromConf(getConfiguration(jobContext).get(COMPRESSION, UNCOMPRESSED.name()));
   }
 
   public static boolean isCompressionSet(JobContext jobContext) {
-    return jobContext.getConfiguration().get(COMPRESSION) != null;
+    return getConfiguration(jobContext).get(COMPRESSION) != null;
+  }
+
+  public static void setValidation(JobContext jobContext, boolean validating) {
+    getConfiguration(jobContext).setBoolean(VALIDATION, validating);
+  }
+
+  public static boolean getValidation(JobContext jobContext) {
+    return getConfiguration(jobContext).getBoolean(VALIDATION, false);
   }
 
   private WriteSupport<T> writeSupport;
@@ -162,7 +188,7 @@
   @SuppressWarnings("unchecked") // writeSupport instantiation
   public RecordWriter<Void, T> getRecordWriter(TaskAttemptContext taskAttemptContext, Path file)
         throws IOException, InterruptedException {
-    final Configuration conf = taskAttemptContext.getConfiguration();
+    final Configuration conf = getConfiguration(taskAttemptContext);
     CodecFactory codecFactory = new CodecFactory(conf);
     int blockSize = getBlockSize(taskAttemptContext);
     if (INFO) LOG.info("Parquet block size to " + blockSize);
@@ -194,14 +220,25 @@
       codec = CompressionCodecName.UNCOMPRESSED;
     }
     if (INFO) LOG.info("Compression: " + codec.name());
-    extension += codec.getExtension();
+    extension = codec.getExtension() + extension;
     if (file == null) {
       file = getDefaultWorkFile(taskAttemptContext, extension);
     }
+    boolean enableDictionary = getEnableDictionary(taskAttemptContext);
     WriteContext init = writeSupport.init(conf);
     ParquetFileWriter w = new ParquetFileWriter(conf, init.getSchema(), file);
     w.start();
-    return new ParquetRecordWriter<T>(w, writeSupport, init.getSchema(), init.getExtraMetaData(), blockSize, pageSize, codecFactory.getCompressor(codec, pageSize));
+    boolean validating = getValidation(taskAttemptContext);
+    if (INFO) LOG.info("Validation is " + (validating ? "on" : "off"));
+    return new ParquetRecordWriter<T>(
+        w,
+        writeSupport,
+        init.getSchema(),
+        init.getExtraMetaData(),
+        blockSize, pageSize,
+        codecFactory.getCompressor(codec, pageSize),
+        enableDictionary,
+        validating);
   }
 
   @Override
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java
new file mode 100644
index 0000000..d3e196a
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetReader.java
@@ -0,0 +1,86 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+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.filter.RecordFilter;
+import parquet.filter.UnboundRecordFilter;
+import parquet.hadoop.api.ReadSupport;
+import parquet.hadoop.api.ReadSupport.ReadContext;
+import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.metadata.FileMetaData;
+import parquet.schema.MessageType;
+
+/**
+ * Read records from a Parquet file.
+ */
+public class ParquetReader<T> implements Closeable {
+
+  private ParquetRecordReader<T> reader;
+
+  public ParquetReader(Path file, ReadSupport<T> readSupport) throws IOException {
+    this(file, readSupport, null);
+  }
+
+  public ParquetReader(Path file, ReadSupport<T> readSupport, UnboundRecordFilter filter) throws IOException {
+    Configuration conf = new Configuration();
+
+    FileSystem fs = FileSystem.get(conf);
+    List<FileStatus> statuses = Arrays.asList(fs.listStatus(file));
+    List<Footer> footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(conf, statuses);
+    Footer footer = footers.get(0); // TODO: check only one
+
+    List<BlockMetaData> blocks = footer.getParquetMetadata().getBlocks();
+    FileMetaData fileMetaData = footer.getParquetMetadata().getFileMetaData();
+    // TODO: this assumes all files have the same schema
+    MessageType schema = fileMetaData.getSchema();
+    Map<String, String> extraMetadata = fileMetaData.getKeyValueMetaData();
+    final ReadContext readContext = readSupport.init(conf, extraMetadata, schema);
+    reader = new ParquetRecordReader<T>(readSupport, filter);
+    ParquetInputSplit inputSplit =
+        new ParquetInputSplit(
+            file, 0, 0, null, blocks,
+            schema.toString(),
+            readContext.getRequestedSchema().toString(),
+            schema.toString(),
+            extraMetadata,
+            readContext.getReadSupportMetadata());
+    reader.initialize(inputSplit, conf);
+  }
+
+  public T read() throws IOException {
+    try {
+      return reader.nextKeyValue() ? reader.getCurrentValue() : null;
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordReader.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordReader.java
index a2998fc..a8ccb02 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordReader.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordReader.java
@@ -15,6 +15,7 @@
  */
 package parquet.hadoop;
 
+import static java.lang.String.format;
 import static parquet.Log.DEBUG;
 
 import java.io.IOException;
@@ -29,10 +30,14 @@
 import parquet.Log;
 import parquet.column.ColumnDescriptor;
 import parquet.column.page.PageReadStore;
+import parquet.filter.RecordFilter;
+import parquet.filter.UnboundRecordFilter;
 import parquet.hadoop.api.ReadSupport;
 import parquet.hadoop.metadata.BlockMetaData;
+import parquet.hadoop.util.ContextUtil;
 import parquet.io.ColumnIOFactory;
 import parquet.io.MessageColumnIO;
+import parquet.io.ParquetDecodingException;
 import parquet.io.api.RecordMaterializer;
 import parquet.schema.GroupType;
 import parquet.schema.MessageType;
@@ -54,6 +59,7 @@
   private final ColumnIOFactory columnIOFactory = new ColumnIOFactory();
 
   private MessageType requestedSchema;
+  private MessageType fileSchema;
   private int columnCount;
   private final ReadSupport<T> readSupport;
 
@@ -62,8 +68,10 @@
   private T currentValue;
   private long total;
   private int current = 0;
+  private int currentBlock = -1;
   private ParquetFileReader reader;
   private parquet.io.RecordReader<T> recordReader;
+  private UnboundRecordFilter recordFilter;
 
   private long totalTimeSpentReadingBytes;
   private long totalTimeSpentProcessingRecords;
@@ -71,15 +79,20 @@
 
   private long totalCountLoadedSoFar = 0;
 
-
+  /**
+   * @param readSupport Object which helps reads files of the given tye, e.g. Thrift, Avro.
+   */
+  public ParquetRecordReader(ReadSupport<T> readSupport) {
+    this(readSupport, null);
+  }
 
   /**
-   *
-   * @param requestedSchema the requested schema (a subset of the original schema) for record projection
-   * @param readSupportClass
+   * @param readSupport Object which helps reads files of the given tye, e.g. Thrift, Avro.
+   * @param filter Optional filter for only returning matching records.
    */
-  ParquetRecordReader(ReadSupport<T> readSupport) {
+  public ParquetRecordReader(ReadSupport<T> readSupport, UnboundRecordFilter filter) {
     this.readSupport = readSupport;
+    this.recordFilter = filter;
   }
 
   private void checkRead() throws IOException {
@@ -104,10 +117,11 @@
       totalTimeSpentReadingBytes += timeSpentReading;
       LOG.info("block read in memory in " + timeSpentReading + " ms. row count = " + pages.getRowCount());
       if (Log.DEBUG) LOG.debug("initializing Record assembly with requested schema " + requestedSchema);
-      MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema);
-      recordReader = columnIO.getRecordReader(pages, recordConverter);
+      MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema, fileSchema);
+      recordReader = columnIO.getRecordReader(pages, recordConverter, recordFilter);
       startedAssemblingCurrentBlockAt = System.currentTimeMillis();
       totalCountLoadedSoFar += pages.getRowCount();
+      ++ currentBlock;
     }
   }
 
@@ -148,17 +162,22 @@
    * {@inheritDoc}
    */
   @Override
-  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+  public void initialize(InputSplit inputSplit, TaskAttemptContext context)
       throws IOException, InterruptedException {
-    Configuration configuration = taskAttemptContext.getConfiguration();
+    initialize(inputSplit, ContextUtil.getConfiguration(context));
+  }
+
+  public void initialize(InputSplit inputSplit, Configuration configuration)
+      throws IOException {
     ParquetInputSplit parquetInputSplit = (ParquetInputSplit)inputSplit;
     this.requestedSchema = MessageTypeParser.parseMessageType(parquetInputSplit.getRequestedSchema());
+    this.fileSchema = MessageTypeParser.parseMessageType(parquetInputSplit.getFileSchema());
     this.columnCount = this.requestedSchema.getPaths().size();
     this.recordConverter = readSupport.prepareForRead(
         configuration,
         parquetInputSplit.getExtraMetadata(),
         MessageTypeParser.parseMessageType(parquetInputSplit.getSchema()),
-        new ReadSupport.ReadContext(requestedSchema));
+        new ReadSupport.ReadContext(requestedSchema, parquetInputSplit.getReadSupportMetadata()));
 
     Path path = parquetInputSplit.getPath();
     List<BlockMetaData> blocks = parquetInputSplit.getBlocks();
@@ -191,10 +210,14 @@
   @Override
   public boolean nextKeyValue() throws IOException, InterruptedException {
     if (current < total) {
-      checkRead();
-      currentValue = recordReader.read();
-      if (DEBUG) LOG.debug("read value: " + currentValue);
-      current ++;
+      try {
+        checkRead();
+        currentValue = recordReader.read();
+        if (DEBUG) LOG.debug("read value: " + currentValue);
+        current ++;
+      } catch (RuntimeException e) {
+        throw new ParquetDecodingException(format("Can not read value at %d in block %d", current, currentBlock), e);
+      }
       return true;
     }
     return false;
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java
index 81c2489..5ef1ba2 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetRecordWriter.java
@@ -15,6 +15,9 @@
  */
 package parquet.hadoop;
 
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+
 import java.io.IOException;
 import java.util.Map;
 
@@ -42,6 +45,8 @@
 public class ParquetRecordWriter<T> extends RecordWriter<Void, T> {
   private static final Log LOG = Log.getLog(ParquetRecordWriter.class);
 
+  private static final int MINIMUM_BUFFER_SIZE = 64 * 1024;
+
   private final ParquetFileWriter w;
   private final WriteSupport<T> writeSupport;
   private final MessageType schema;
@@ -49,6 +54,8 @@
   private final int blockSize;
   private final int pageSize;
   private final BytesCompressor compressor;
+  private final boolean enableDictionary;
+  private final boolean validating;
 
   private long recordCount = 0;
   private long recordCountForNextMemCheck = 100;
@@ -57,6 +64,7 @@
   private ColumnChunkPageWriteStore pageStore;
 
 
+
   /**
    *
    * @param w the file to write to
@@ -66,7 +74,7 @@
    * @param blockSize the size of a block in the file (this will be approximate)
    * @param codec the codec used to compress
    */
-  public ParquetRecordWriter(ParquetFileWriter w, WriteSupport<T> writeSupport, MessageType schema,  Map<String, String> extraMetaData, int blockSize, int pageSize, BytesCompressor compressor) {
+  public ParquetRecordWriter(ParquetFileWriter w, WriteSupport<T> writeSupport, MessageType schema,  Map<String, String> extraMetaData, int blockSize, int pageSize, BytesCompressor compressor, boolean enableDictionary, boolean validating) {
     if (writeSupport == null) {
       throw new NullPointerException("writeSupport");
     }
@@ -77,13 +85,22 @@
     this.blockSize = blockSize;
     this.pageSize = pageSize;
     this.compressor = compressor;
+    this.enableDictionary = enableDictionary;
+    this.validating = validating;
     initStore();
   }
 
   private void initStore() {
-    pageStore = new ColumnChunkPageWriteStore(compressor, schema);
-    store = new ColumnWriteStoreImpl(pageStore, pageSize);
-    MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
+    // we don't want this number to be too small
+    // ideally we divide the block equally across the columns
+    // it is unlikely all columns are going to be the same size.
+    int initialBlockBufferSize = max(MINIMUM_BUFFER_SIZE, blockSize / schema.getColumns().size() / 5);
+    pageStore = new ColumnChunkPageWriteStore(compressor, schema, initialBlockBufferSize);
+    // we don't want this number to be too small either
+    // ideally, slightly bigger than the page size, but not bigger than the block buffer
+    int initialPageBufferSize = max(MINIMUM_BUFFER_SIZE, min(pageSize + pageSize / 10, initialBlockBufferSize));
+    store = new ColumnWriteStoreImpl(pageStore, pageSize, initialPageBufferSize, enableDictionary);
+    MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(schema);
     writeSupport.prepareForWrite(columnIO.getRecordWriter(store));
   }
 
@@ -118,7 +135,7 @@
       } else {
         float recordSize = (float) memSize / recordCount;
         recordCountForNextMemCheck = Math.max(100, (recordCount + (long)(blockSize / recordSize)) / 2); // will check halfway
-        LOG.info("Checked mem at " + recordCount + " will check again at: " + recordCountForNextMemCheck);
+        LOG.debug("Checked mem at " + recordCount + " will check again at: " + recordCountForNextMemCheck);
       }
     }
   }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
new file mode 100644
index 0000000..4a10f5c
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
@@ -0,0 +1,116 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import parquet.hadoop.api.WriteSupport;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.schema.MessageType;
+
+/**
+ * Write records to a Parquet file.
+ */
+public class ParquetWriter<T> implements Closeable {
+
+  public static final int DEFAULT_BLOCK_SIZE = 50*1024*1024;
+  public static final int DEFAULT_PAGE_SIZE = 1*1024*1024;
+
+  private final ParquetRecordWriter<T> writer;
+
+  /**
+   * Create a new ParquetWriter.
+   * (with dictionary encoding disabled and validation off)
+   * @see ParquetWriter#ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, boolean)
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold
+   * @throws IOException
+   */
+  public ParquetWriter(Path file, WriteSupport<T> writeSupport, CompressionCodecName compressionCodecName, int blockSize, int pageSize) throws IOException {
+    this(file, writeSupport, compressionCodecName, blockSize, pageSize, false, false);
+  }
+
+  /**
+   * Create a new ParquetWriter.
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold
+   * @param enableDictionary to turn dictionary encoding on
+   * @param validating to turn on validation using the schema
+   * @throws IOException
+   */
+  public ParquetWriter(
+      Path file,
+      WriteSupport<T> writeSupport,
+      CompressionCodecName compressionCodecName,
+      int blockSize,
+      int pageSize,
+      boolean enableDictionary,
+      boolean validating) throws IOException {
+    Configuration conf = new Configuration();
+
+    WriteSupport.WriteContext writeContext = writeSupport.init(conf);
+    MessageType schema = writeContext.getSchema();
+
+    ParquetFileWriter fileWriter = new ParquetFileWriter(conf, schema, file);
+    fileWriter.start();
+
+    CodecFactory codecFactory = new CodecFactory(conf);
+    CodecFactory.BytesCompressor compressor =	codecFactory.getCompressor(compressionCodecName, 0);
+    this.writer = new ParquetRecordWriter<T>(fileWriter, writeSupport, schema, writeContext.getExtraMetaData(), blockSize, pageSize, compressor, enableDictionary, validating);
+
+  }
+
+  /**
+   * Create a new ParquetWriter.  The default block size is 50 MB.The default
+   * page size is 1 MB.  Default compression is no compression. Dictionary encoding is disabled.
+   *
+   * @param file the file to create
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @throws IOException
+   */
+  public ParquetWriter(Path file, WriteSupport<T> writeSupport) throws IOException {
+    this(file, writeSupport, CompressionCodecName.UNCOMPRESSED, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
+  }
+
+  public void write(T object) throws IOException {
+    try {
+      writer.write(null, object);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      writer.close(null);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java b/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java
index 5761f09..bd42da3 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/PrintFooter.java
@@ -26,6 +26,7 @@
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -39,6 +40,7 @@
 import org.apache.hadoop.fs.PathFilter;
 
 import parquet.column.ColumnDescriptor;
+import parquet.column.Encoding;
 import parquet.hadoop.metadata.BlockMetaData;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 import parquet.hadoop.metadata.ParquetMetadata;
@@ -168,7 +170,8 @@
             desc,
             columnMetaData.getValueCount(),
             columnMetaData.getTotalSize(),
-            columnMetaData.getTotalUncompressedSize());
+            columnMetaData.getTotalUncompressedSize(),
+            columnMetaData.getEncodings());
       }
     }
   }
@@ -230,32 +233,34 @@
     Stats valueCountStats = new Stats();
     Stats allStats = new Stats();
     Stats uncStats = new Stats();
+    Set<Encoding> encodings = new TreeSet<Encoding>();
     int blocks = 0;
 
-    public void add(long valueCount, long size, long uncSize) {
+    public void add(long valueCount, long size, long uncSize, List<Encoding> encodings) {
       ++blocks;
       valueCountStats.add(valueCount);
       allStats.add(size);
       uncStats.add(uncSize);
+      this.encodings.addAll(encodings);
     }
 
     @Override
     public String toString() {
       long raw = uncStats.total;
       long compressed = allStats.total;
-      return allStats.toString(blocks) + " (raw data: " + humanReadable(raw) + (raw == 0 ? "" : " saving " + (raw - compressed)*100/raw + "%") + ")\n"
+      return encodings + " " + allStats.toString(blocks) + " (raw data: " + humanReadable(raw) + (raw == 0 ? "" : " saving " + (raw - compressed)*100/raw + "%") + ")\n"
       + "  values: "+valueCountStats.toString(blocks) + "\n"
       + "  uncompressed: "+uncStats.toString(blocks);
     }
 
   }
 
-  private static void add(ColumnDescriptor desc, long valueCount, long size, long uncSize) {
+  private static void add(ColumnDescriptor desc, long valueCount, long size, long uncSize, List<Encoding> encodings) {
     ColStats colStats = stats.get(desc);
     if (colStats == null) {
       colStats = new ColStats();
       stats.put(desc, colStats);
     }
-    colStats.add(valueCount, size, uncSize);
+    colStats.add(valueCount, size, uncSize, encodings);
   }
 }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/api/ReadSupport.java b/parquet-hadoop/src/main/java/parquet/hadoop/api/ReadSupport.java
index 484c636..4184656 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/api/ReadSupport.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/api/ReadSupport.java
@@ -21,6 +21,7 @@
 
 import parquet.io.api.RecordMaterializer;
 import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
 
 /**
  * Abstraction used by the {@link parquet.hadoop.ParquetInputFormat} to materialize records
@@ -32,6 +33,11 @@
 abstract public class ReadSupport<T> {
 
   /**
+   * configuration key for a parquet read projection schema
+   */
+	public static final String PARQUET_READ_SCHEMA = "parquet.read.schema";
+  
+  /**
    * information to read the file
    *
    * @author Julien Le Dem
@@ -39,20 +45,41 @@
    */
   public static final class ReadContext {
     private final MessageType requestedSchema;
+    private final Map<String, String> readSupportMetadata;
 
+    /**
+     * @param requestedSchema the schema requested by the user. Can not be null.
+     */
     public ReadContext(MessageType requestedSchema) {
+      this(requestedSchema, null);
+    }
+
+    /**
+     * @param requestedSchema the schema requested by the user. Can not be null.
+     * @param readSupportMetadata metadata specific to the ReadSupport implementation. Will be available in the prepareForRead phase.
+     */
+    public ReadContext(MessageType requestedSchema, Map<String, String> readSupportMetadata) {
       super();
       if (requestedSchema == null) {
         throw new NullPointerException("requestedSchema");
       }
       this.requestedSchema = requestedSchema;
+      this.readSupportMetadata = readSupportMetadata;
     }
+
     /**
      * @return the schema of the file
      */
     public MessageType getRequestedSchema() {
       return requestedSchema;
     }
+
+    /**
+     * @return metadata specific to the ReadSupport implementation
+     */
+    public Map<String, String> getReadSupportMetadata() {
+      return readSupportMetadata;
+    }
   }
 
   /**
@@ -61,7 +88,6 @@
    * @param configuration the job configuration
    * @param keyValueMetaData the app specific metadata from the file
    * @param fileSchema the schema of the file
-   * @param requestedSchema the schema requested by the user
    * @return the readContext that defines how to read the file
    */
   abstract public ReadContext init(
@@ -83,5 +109,21 @@
       Map<String, String> keyValueMetaData,
       MessageType fileSchema,
       ReadContext readContext);
+  
+  /**
+   * attempts to validate and construct a {@link MessageType} from a read projection schema
+   * @param fileMessageType the typed schema of the source
+   * @param partialReadSchemaString the requested projection schema 
+   * @return the typed schema that should be used to read
+   */
+  public static MessageType getSchemaForRead(MessageType fileMessageType, String partialReadSchemaString) {
+  	MessageType forRead = fileMessageType;
+    if (partialReadSchemaString != null) {
+      MessageType requestedMessageType = MessageTypeParser.parseMessageType(partialReadSchemaString);
+      fileMessageType.checkContains(requestedMessageType);
+      forRead = requestedMessageType;
+    }
+    return forRead;
+  }
 
 }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyCodec.java b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyCodec.java
new file mode 100644
index 0000000..70c4f11
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyCodec.java
@@ -0,0 +1,104 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.codec;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionInputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.CompressorStream;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DecompressorStream;
+
+/**
+ * Snappy compression codec for Parquet.  We do not use the default hadoop
+ * one since that codec adds a blocking structure around the base snappy compression
+ * algorithm.  This is useful for hadoop to minimize the size of compression blocks
+ * for their file formats (e.g. SequenceFile) but is undesirable for Parquet since
+ * we already have the data page which provides that.
+ */
+public class SnappyCodec implements Configurable, CompressionCodec {
+  private Configuration conf;
+  // Hadoop config for how big to make intermediate buffers.
+  private final String BUFFER_SIZE_CONFIG = "io.file.buffer.size";
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Compressor createCompressor() {
+    return new SnappyCompressor();
+  }
+
+  @Override
+  public Decompressor createDecompressor() {
+    return new SnappyDecompressor();
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(InputStream stream)
+      throws IOException {
+    return createInputStream(stream, createDecompressor());
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(InputStream stream,
+      Decompressor decompressor) throws IOException {
+    return new DecompressorStream(stream, decompressor,
+        conf.getInt(BUFFER_SIZE_CONFIG, 4*1024));
+  }
+
+  @Override
+  public CompressionOutputStream createOutputStream(OutputStream stream)
+      throws IOException {
+    return createOutputStream(stream, createCompressor());
+  }
+
+  @Override
+  public CompressionOutputStream createOutputStream(OutputStream stream,
+      Compressor compressor) throws IOException {
+    return new CompressorStream(stream, compressor, 
+        conf.getInt(BUFFER_SIZE_CONFIG, 4*1024));
+  }
+
+  @Override
+  public Class<? extends Compressor> getCompressorType() {
+    return SnappyCompressor.class;
+  }
+
+  @Override
+  public Class<? extends Decompressor> getDecompressorType() {
+    return SnappyDecompressor.class;
+  }
+
+  @Override
+  public String getDefaultExtension() {
+    return ".snappy";
+  }  
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyCompressor.java b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyCompressor.java
new file mode 100644
index 0000000..61520d5
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyCompressor.java
@@ -0,0 +1,156 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.codec;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.Compressor;
+import org.xerial.snappy.Snappy;
+
+import parquet.Preconditions;
+
+/**
+ * This class is a wrapper around the snappy compressor. It always consumes the
+ * entire input in setInput and compresses it as one compressed block.
+ */
+public class SnappyCompressor implements Compressor {
+  // Buffer for compressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for uncompressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private long bytesRead = 0L;
+  private long bytesWritten = 0L;
+  private boolean finishCalled = false;
+
+  /**
+   * Fills specified buffer with compressed data. Returns actual number
+   * of bytes of compressed data. A return value of 0 indicates that
+   * needsInput() should be called in order to determine if more input
+   * data is required.
+   *
+   * @param buffer   Buffer for the compressed data
+   * @param off Start offset of the data
+   * @param len Size of the buffer
+   * @return The actual number of bytes of compressed data.
+   */
+  @Override
+  public synchronized int compress(byte[] buffer, int off, int len) throws IOException {
+    SnappyUtil.validateBuffer(buffer, off, len);
+
+    if (needsInput()) {
+      // No buffered output bytes and no input to consume, need more input
+      return 0;
+    }
+
+    if (!outputBuffer.hasRemaining()) {
+      // There is uncompressed input, compress it now
+      int maxOutputSize = Snappy.maxCompressedLength(inputBuffer.position());
+      if (maxOutputSize > outputBuffer.capacity()) {
+        outputBuffer = ByteBuffer.allocateDirect(maxOutputSize);
+      }
+      // Reset the previous outputBuffer
+      outputBuffer.clear();
+      inputBuffer.limit(inputBuffer.position());
+      inputBuffer.position(0);
+
+      int size = Snappy.compress(inputBuffer, outputBuffer);
+      outputBuffer.limit(size);
+      inputBuffer.limit(0);
+      inputBuffer.rewind();
+    }
+
+    // Return compressed output up to 'len'
+    int numBytes = Math.min(len, outputBuffer.remaining());
+    outputBuffer.get(buffer, off, numBytes);    
+    bytesWritten += numBytes;
+    return numBytes;	    
+  }
+
+  @Override
+  public synchronized void setInput(byte[] buffer, int off, int len) {  
+    SnappyUtil.validateBuffer(buffer, off, len);
+    
+    Preconditions.checkArgument(!outputBuffer.hasRemaining(), 
+        "Output buffer should be empty. Caller must call compress()");
+
+    if (inputBuffer.remaining() < len) {
+      ByteBuffer tmp = ByteBuffer.allocateDirect(inputBuffer.capacity() + len);
+      inputBuffer.rewind();
+      tmp.put(inputBuffer);
+      inputBuffer = tmp;
+    }
+
+    // Append the current bytes to the input buffer
+    inputBuffer.put(buffer, off, len);
+    bytesRead += len;
+  }
+
+  @Override
+  public void end() {
+    // No-op		
+  }
+
+  @Override
+  public void finish() {
+    finishCalled = true;
+  }
+
+  @Override
+  public synchronized boolean finished() {
+    return finishCalled && inputBuffer.position() == 0 && !outputBuffer.hasRemaining();
+  }
+
+  @Override
+  public long getBytesRead() {
+    return bytesRead;
+  }
+
+  @Override
+  public long getBytesWritten() {
+    return bytesWritten;
+  }
+
+  @Override
+  // We want to compress all the input in one go so we always need input until it is
+  // all consumed.
+  public synchronized boolean needsInput() {
+    return !finishCalled;
+  }
+
+  @Override
+  public void reinit(Configuration c) {
+    reset();		
+  }
+
+  @Override
+  public synchronized void reset() {
+    finishCalled = false;
+    bytesRead = bytesWritten = 0;
+    inputBuffer.clear();
+    inputBuffer.limit(0);
+    outputBuffer.clear();
+    outputBuffer.limit(0);		
+  }
+
+  @Override
+  public void setDictionary(byte[] dictionary, int off, int len) {
+    // No-op		
+  }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyDecompressor.java b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyDecompressor.java
new file mode 100644
index 0000000..f1c9e2a
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyDecompressor.java
@@ -0,0 +1,149 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.codec;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.io.compress.Decompressor;
+import org.xerial.snappy.Snappy;
+
+import parquet.Preconditions;
+
+public class SnappyDecompressor implements Decompressor {
+  // Buffer for uncompressed output. This buffer grows as necessary.
+  private ByteBuffer outputBuffer = ByteBuffer.allocateDirect(0);
+
+  // Buffer for compressed input. This buffer grows as necessary.
+  private ByteBuffer inputBuffer = ByteBuffer.allocateDirect(0);
+
+  private boolean finished;
+  
+  /**
+   * Fills specified buffer with uncompressed data. Returns actual number
+   * of bytes of uncompressed data. A return value of 0 indicates that
+   * {@link #needsInput()} should be called in order to determine if more
+   * input data is required.
+   *
+   * @param buffer   Buffer for the compressed data
+   * @param off Start offset of the data
+   * @param len Size of the buffer
+   * @return The actual number of bytes of uncompressed data.
+   * @throws IOException
+   */
+  @Override
+  public synchronized int decompress(byte[] buffer, int off, int len) throws IOException {
+    SnappyUtil.validateBuffer(buffer, off, len);
+
+    if (needsInput()) {    	
+      // No buffered output bytes and no input to consume, need more input
+      return 0;
+    }
+    
+    if (!outputBuffer.hasRemaining()) {
+      Preconditions.checkArgument(inputBuffer.position() == 0, "Invalid position of 0.");
+      Preconditions.checkArgument(outputBuffer.position() == 0, "Invalid position of 0.");
+      // There is compressed input, decompress it now.
+      int decompressedSize = Snappy.uncompressedLength(inputBuffer);
+      if (decompressedSize > outputBuffer.capacity()) {
+        outputBuffer = ByteBuffer.allocateDirect(decompressedSize);
+      }
+
+      // Reset the previous outputBuffer (i.e. set position to 0)
+      outputBuffer.clear();
+      int size = Snappy.uncompress(inputBuffer, outputBuffer);
+      outputBuffer.limit(size);
+      // We've compressed the entire input, reset the input now
+      inputBuffer.clear();
+      inputBuffer.limit(0);
+      finished = true;
+    }
+
+    // Return compressed output up to 'len'
+
+    int numBytes = Math.min(len, outputBuffer.remaining());
+    outputBuffer.get(buffer, off, numBytes);
+    return numBytes;	    
+  }
+
+  /**
+   * Sets input data for decompression.
+   * This should be called if and only if {@link #needsInput()} returns
+   * <code>true</code> indicating that more input data is required.
+   * (Both native and non-native versions of various Decompressors require
+   * that the data passed in via <code>b[]</code> remain unmodified until
+   * the caller is explicitly notified--via {@link #needsInput()}--that the
+   * buffer may be safely modified.  With this requirement, an extra
+   * buffer-copy can be avoided.)
+   *
+   * @param buffer   Input data
+   * @param off Start offset
+   * @param len Length
+   */
+  @Override
+  public synchronized void setInput(byte[] buffer, int off, int len) {
+    SnappyUtil.validateBuffer(buffer, off, len);
+
+    if (inputBuffer.capacity() < len) {
+      inputBuffer = ByteBuffer.allocateDirect(len);
+    }
+
+    // Clear the input direct buffer and put all the input data there.
+    inputBuffer.clear();
+    inputBuffer.put(buffer, off, len);
+    inputBuffer.rewind();
+    inputBuffer.limit(len);
+  }
+
+  @Override
+  public void end() {
+    // No-op		
+  }
+
+  @Override
+  public boolean finished() {
+    return finished && !outputBuffer.hasRemaining();
+  }
+
+  @Override
+  public int getRemaining() {
+    return 0;
+  }
+
+  @Override
+  public synchronized boolean needsInput() {
+    return !inputBuffer.hasRemaining() && !outputBuffer.hasRemaining();
+  }
+
+  @Override
+  public synchronized void reset() {
+    finished = false;
+    inputBuffer.clear();
+    inputBuffer.limit(0);
+    outputBuffer.clear();
+    outputBuffer.limit(0);		
+  }
+
+  @Override
+  public boolean needsDictionary() {
+    return false;
+  }
+
+  @Override
+  public void setDictionary(byte[] b, int off, int len) {
+    // No-op		
+  }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyUtil.java b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyUtil.java
new file mode 100644
index 0000000..389dcce
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyUtil.java
@@ -0,0 +1,29 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.codec;
+
+import parquet.Preconditions;
+
+/**
+ * Utilities for SnappyCompressor and SnappyDecompressor.
+ */
+public class SnappyUtil {
+  public static void validateBuffer(byte[] buffer, int off, int len) {
+    Preconditions.checkNotNull(buffer, "buffer");
+    Preconditions.checkArgument(off >= 0 && len >= 0 && off <= buffer.length - len,
+        "Invalid offset or length. Out of buffer bounds.");
+  }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/example/ExampleOutputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/example/ExampleOutputFormat.java
index c7f793f..127602d 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/example/ExampleOutputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/example/ExampleOutputFormat.java
@@ -20,6 +20,7 @@
 
 import parquet.example.data.Group;
 import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.util.ContextUtil;
 import parquet.schema.MessageType;
 
 /**
@@ -40,7 +41,7 @@
    * @param configuration the job configuration
    */
   public static void setSchema(Job job, MessageType schema) {
-    GroupWriteSupport.setSchema(schema, job.getConfiguration());
+    GroupWriteSupport.setSchema(schema, ContextUtil.getConfiguration(job));
   }
 
   /**
@@ -49,7 +50,7 @@
    * @return the schema
    */
   public static MessageType getSchema(Job job) {
-    return GroupWriteSupport.getSchema(job.getConfiguration());
+    return GroupWriteSupport.getSchema(ContextUtil.getConfiguration(job));
   }
 
   public ExampleOutputFormat() {
diff --git a/parquet-column/src/main/java/parquet/column/values/ValuesType.java b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/Container.java
similarity index 69%
copy from parquet-column/src/main/java/parquet/column/values/ValuesType.java
copy to parquet-hadoop/src/main/java/parquet/hadoop/mapred/Container.java
index 0566303..0ccaf72 100644
--- a/parquet-column/src/main/java/parquet/column/values/ValuesType.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/Container.java
@@ -13,14 +13,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package parquet.column.values;
+package parquet.hadoop.mapred;
 
 /**
- * The different type of values we can store in columns
- *
- * @author Julien Le Dem
- *
+ * A simple container of <T> objects that you can get and set.
+ * @param <T>
  */
-public enum ValuesType {
-  REPETITION_LEVEL, DEFINITION_LEVEL, VALUES;
+public class Container<T> {
+
+  T object;
+
+  public void set(T object) {
+    this.object = object;
+  }
+
+  public T get() {
+    return object;
+  }
+
 }
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetInputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetInputFormat.java
new file mode 100644
index 0000000..4f1eb2c
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/mapred/DeprecatedParquetInputFormat.java
@@ -0,0 +1,204 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Arrays;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputFormat;
+
+import parquet.hadoop.ParquetInputFormat;
+import parquet.hadoop.ParquetInputSplit;
+import parquet.hadoop.ParquetRecordReader;
+import parquet.hadoop.Footer;
+
+@SuppressWarnings("deprecation")
+public class DeprecatedParquetInputFormat<V> extends org.apache.hadoop.mapred.FileInputFormat<Void, Container<V>> {
+
+  protected ParquetInputFormat<V> realInputFormat = new ParquetInputFormat<V>();
+
+  @Override
+  public RecordReader<Void, Container<V>> getRecordReader(InputSplit split, JobConf job,
+                  Reporter reporter) throws IOException {
+    return new RecordReaderWrapper<V>(realInputFormat, split, job, reporter);
+  }
+
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+      List<Footer> footers = realInputFormat.getFooters(job, Arrays.asList(super.listStatus(job)));
+      List<ParquetInputSplit> splits = realInputFormat.getSplits(job, footers);
+
+      if (splits == null) {
+        return null;
+      }
+
+      InputSplit[] resultSplits = new InputSplit[splits.size()];
+      int i = 0;
+      for (ParquetInputSplit split : splits) {
+          resultSplits[i++] = new ParquetInputSplitWrapper(split);
+      }
+
+      return resultSplits;
+  }
+
+  private static class RecordReaderWrapper<V> implements RecordReader<Void, Container<V>> {
+
+    private ParquetRecordReader<V> realReader;
+    private long splitLen; // for getPos()
+
+    private Container<V> valueContainer = null;
+
+    private boolean firstRecord = false;
+    private boolean eof = false;
+
+    public RecordReaderWrapper(ParquetInputFormat<V> newInputFormat,
+                               InputSplit oldSplit,
+                               JobConf oldJobConf,
+                               Reporter reporter) throws IOException {
+
+      splitLen = oldSplit.getLength();
+
+      try {
+        realReader = new ParquetRecordReader<V>(newInputFormat.getReadSupport(oldJobConf));
+        realReader.initialize(((ParquetInputSplitWrapper)oldSplit).realSplit, oldJobConf);
+
+        // read once to gain access to key and value objects
+        if (realReader.nextKeyValue()) {
+          firstRecord = true;
+          valueContainer = new Container<V>();
+          valueContainer.set(realReader.getCurrentValue());
+
+        } else {
+          eof = true;
+        }
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new IOException(e);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      realReader.close();
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public Container<V> createValue() {
+      return valueContainer;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return (long) (splitLen * getProgress());
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      try {
+        return realReader.getProgress();
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new IOException(e);
+      }
+    }
+
+    @Override
+    public boolean next(Void key, Container<V> value) throws IOException {
+      if (eof) {
+        return false;
+      }
+
+      if (firstRecord) { // key & value are already read.
+        firstRecord = false;
+        return true;
+      }
+
+      try {
+        if (realReader.nextKeyValue()) {
+          if (value != null) value.set(realReader.getCurrentValue());
+          return true;
+        }
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+
+      eof = true; // strictly not required, just for consistency
+      return false;
+    }
+  }
+
+
+
+  private static class ParquetInputSplitWrapper implements InputSplit {
+
+    ParquetInputSplit realSplit;
+
+
+    @SuppressWarnings("unused") // MapReduce instantiates this.
+    public ParquetInputSplitWrapper() {}
+
+    public ParquetInputSplitWrapper(ParquetInputSplit realSplit) {
+      this.realSplit = realSplit;
+    }
+
+    @Override
+    public long getLength() throws IOException {
+      try {
+        return realSplit.getLength();
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new IOException(e);
+      }
+    }
+
+    @Override
+    public String[] getLocations() throws IOException {
+      try {
+        return realSplit.getLocations();
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new IOException(e);
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      realSplit = new ParquetInputSplit();
+      realSplit.readFields(in);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      realSplit.write(out);
+    }
+  }
+}
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/ColumnChunkMetaData.java b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/ColumnChunkMetaData.java
index 5d29112..e1880d1 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/ColumnChunkMetaData.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/ColumnChunkMetaData.java
@@ -37,6 +37,7 @@
   private final List<Encoding> encodings;
 
   private long firstDataPage;
+  private long dictionaryPageOffset;
 
   // for info
   private long valueCount;
@@ -46,7 +47,6 @@
 
 
   /**
-   *
    * @param path column identifier
    * @param type type of the column
    * @param encodings
@@ -71,7 +71,6 @@
   }
 
   /**
-   *
    * @return type of the column
    */
   public PrimitiveTypeName getType() {
@@ -79,7 +78,6 @@
   }
 
   /**
-   *
    * @param dataStart offset in the file where data starts
    */
   public void setFirstDataPageOffset(long firstDataPage) {
@@ -87,7 +85,13 @@
   }
 
   /**
-   *
+   * @param dictionaryPageOffset offset in the file of the dictionary page
+   */
+  public void setDictionaryPageOffset(long dictionaryPageOffset) {
+    this.dictionaryPageOffset = dictionaryPageOffset;
+  }
+
+  /**
    * @return start of the column data offset
    */
   public long getFirstDataPageOffset() {
@@ -95,7 +99,13 @@
   }
 
   /**
-   *
+   * @return the location of the dictionary page if any
+   */
+  public long getDictionaryPageOffset() {
+    return dictionaryPageOffset;
+  }
+
+  /**
    * @param valueCount count of values in this block of the column
    */
   public void setValueCount(long valueCount) {
@@ -103,7 +113,6 @@
   }
 
   /**
-   *
    * @return count of values in this block of the column
    */
   public long getValueCount() {
@@ -123,6 +132,7 @@
   public void setTotalUncompressedSize(long totalUncompressedSize) {
     this.totalUncompressedSize = totalUncompressedSize;
   }
+
   /**
    * @return the totalSize
    */
@@ -138,7 +148,6 @@
   }
 
   /**
-   *
    * @return all the encodings used in this column
    */
   public List<Encoding> getEncodings() {
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java
index 59e57a7..195d593 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/CompressionCodecName.java
@@ -19,7 +19,7 @@
 
 public enum CompressionCodecName {
   UNCOMPRESSED(null, CompressionCodec.UNCOMPRESSED, ""),
-  SNAPPY("org.apache.hadoop.io.compress.SnappyCodec", CompressionCodec.SNAPPY, ".snappy"),
+  SNAPPY("parquet.hadoop.codec.SnappyCodec", CompressionCodec.SNAPPY, ".snappy"),
   GZIP("org.apache.hadoop.io.compress.GzipCodec", CompressionCodec.GZIP, ".gz"),
   LZO("com.hadoop.compression.lzo.LzoCodec", CompressionCodec.LZO, ".lzo");
 
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/FileMetaData.java b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/FileMetaData.java
index 4adc241..6bd5842 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/metadata/FileMetaData.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/metadata/FileMetaData.java
@@ -34,11 +34,14 @@
 
   private final Map<String, String> keyValueMetaData;
 
+  private final String createdBy;
+
   /**
    * @param schema the schema for the file
    * @param keyValueMetaData the app specific metadata
+   * @param createdBy the description of the library that created the file
    */
-  public FileMetaData(MessageType schema, Map<String, String> keyValueMetaData) {
+  public FileMetaData(MessageType schema, Map<String, String> keyValueMetaData, String createdBy) {
     super();
     if (schema == null) {
       throw new NullPointerException("schema");
@@ -48,6 +51,7 @@
     }
     this.schema = schema;
     this.keyValueMetaData = keyValueMetaData;
+    this.createdBy = createdBy;
   }
 
   /**
@@ -69,4 +73,11 @@
     return keyValueMetaData;
   }
 
+  /**
+   * @return the description of the library that created the file
+   */
+  public String getCreatedBy() {
+    return createdBy;
+  }
+
 }
\ No newline at end of file
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/util/ContextUtil.java b/parquet-hadoop/src/main/java/parquet/hadoop/util/ContextUtil.java
new file mode 100644
index 0000000..7acbe60
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/util/ContextUtil.java
@@ -0,0 +1,236 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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 java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.StatusReporter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+/*
+ * This is based on ContextFactory.java from hadoop-2.0.x sources.
+ */
+
+/**
+ * Utility methods to allow applications to deal with inconsistencies between
+ * MapReduce Context Objects API between hadoop-0.20 and later versions.
+ */
+public class ContextUtil {
+
+  private static final boolean useV21;
+
+  private static final Constructor<?> JOB_CONTEXT_CONSTRUCTOR;
+  private static final Constructor<?> TASK_CONTEXT_CONSTRUCTOR;
+  private static final Constructor<?> MAP_CONTEXT_CONSTRUCTOR;
+  private static final Constructor<?> MAP_CONTEXT_IMPL_CONSTRUCTOR;
+  private static final Constructor<?> GENERIC_COUNTER_CONSTRUCTOR;
+
+  private static final Field READER_FIELD;
+  private static final Field WRITER_FIELD;
+  private static final Field OUTER_MAP_FIELD;
+  private static final Field WRAPPED_CONTEXT_FIELD;
+
+  private static final Method GET_CONFIGURATION_METHOD;
+
+  static {
+    boolean v21 = true;
+    final String PACKAGE = "org.apache.hadoop.mapreduce";
+    try {
+      Class.forName(PACKAGE + ".task.JobContextImpl");
+    } catch (ClassNotFoundException cnfe) {
+      v21 = false;
+    }
+    useV21 = v21;
+    Class<?> jobContextCls;
+    Class<?> taskContextCls;
+    Class<?> taskIOContextCls;
+    Class<?> mapCls;
+    Class<?> mapContextCls;
+    Class<?> innerMapContextCls;
+    Class<?> genericCounterCls;
+    try {
+      if (v21) {
+        jobContextCls =
+            Class.forName(PACKAGE+".task.JobContextImpl");
+        taskContextCls =
+            Class.forName(PACKAGE+".task.TaskAttemptContextImpl");
+        taskIOContextCls =
+            Class.forName(PACKAGE+".task.TaskInputOutputContextImpl");
+        mapContextCls = Class.forName(PACKAGE + ".task.MapContextImpl");
+        mapCls = Class.forName(PACKAGE + ".lib.map.WrappedMapper");
+        innerMapContextCls =
+            Class.forName(PACKAGE+".lib.map.WrappedMapper$Context");
+        genericCounterCls = Class.forName(PACKAGE+".counters.GenericCounter");
+      } else {
+        jobContextCls =
+            Class.forName(PACKAGE+".JobContext");
+        taskContextCls =
+            Class.forName(PACKAGE+".TaskAttemptContext");
+        taskIOContextCls =
+            Class.forName(PACKAGE+".TaskInputOutputContext");
+        mapContextCls = Class.forName(PACKAGE + ".MapContext");
+        mapCls = Class.forName(PACKAGE + ".Mapper");
+        innerMapContextCls =
+            Class.forName(PACKAGE+".Mapper$Context");
+        genericCounterCls =
+            Class.forName("org.apache.hadoop.mapred.Counters$Counter");
+      }
+    } catch (ClassNotFoundException e) {
+      throw new IllegalArgumentException("Can't find class", e);
+    }
+    try {
+      JOB_CONTEXT_CONSTRUCTOR =
+          jobContextCls.getConstructor(Configuration.class, JobID.class);
+      JOB_CONTEXT_CONSTRUCTOR.setAccessible(true);
+      TASK_CONTEXT_CONSTRUCTOR =
+          taskContextCls.getConstructor(Configuration.class,
+              TaskAttemptID.class);
+      TASK_CONTEXT_CONSTRUCTOR.setAccessible(true);
+      GENERIC_COUNTER_CONSTRUCTOR =
+          genericCounterCls.getDeclaredConstructor(String.class,
+              String.class,
+              Long.TYPE);
+      GENERIC_COUNTER_CONSTRUCTOR.setAccessible(true);
+
+      if (useV21) {
+        MAP_CONTEXT_CONSTRUCTOR =
+            innerMapContextCls.getConstructor(mapCls,
+                MapContext.class);
+        MAP_CONTEXT_IMPL_CONSTRUCTOR =
+            mapContextCls.getDeclaredConstructor(Configuration.class,
+                TaskAttemptID.class,
+                RecordReader.class,
+                RecordWriter.class,
+                OutputCommitter.class,
+                StatusReporter.class,
+                InputSplit.class);
+        MAP_CONTEXT_IMPL_CONSTRUCTOR.setAccessible(true);
+        WRAPPED_CONTEXT_FIELD =
+            innerMapContextCls.getDeclaredField("mapContext");
+        WRAPPED_CONTEXT_FIELD.setAccessible(true);
+
+      } else {
+        MAP_CONTEXT_CONSTRUCTOR =
+            innerMapContextCls.getConstructor(mapCls,
+                Configuration.class,
+                TaskAttemptID.class,
+                RecordReader.class,
+                RecordWriter.class,
+                OutputCommitter.class,
+                StatusReporter.class,
+                InputSplit.class);
+        MAP_CONTEXT_IMPL_CONSTRUCTOR = null;
+        WRAPPED_CONTEXT_FIELD = null;
+      }
+      MAP_CONTEXT_CONSTRUCTOR.setAccessible(true);
+      READER_FIELD = mapContextCls.getDeclaredField("reader");
+      READER_FIELD.setAccessible(true);
+      WRITER_FIELD = taskIOContextCls.getDeclaredField("output");
+      WRITER_FIELD.setAccessible(true);
+      OUTER_MAP_FIELD = innerMapContextCls.getDeclaredField("this$0");
+      OUTER_MAP_FIELD.setAccessible(true);
+      GET_CONFIGURATION_METHOD = Class.forName(PACKAGE+".JobContext")
+          .getMethod("getConfiguration");
+    } catch (SecurityException e) {
+      throw new IllegalArgumentException("Can't run constructor ", e);
+    } catch (NoSuchMethodException e) {
+      throw new IllegalArgumentException("Can't find constructor ", e);
+    } catch (NoSuchFieldException e) {
+      throw new IllegalArgumentException("Can't find field ", e);
+    } catch (ClassNotFoundException e) {
+      throw new IllegalArgumentException("Can't find class", e);
+    }
+  }
+
+  /**
+   * Creates JobContext from a JobConf and jobId using the correct constructor
+   * for based on Hadoop version. <code>jobId</code> could be null.
+   */
+  public static JobContext newJobContext(Configuration conf, JobID jobId) {
+    try {
+      return (JobContext)
+          JOB_CONTEXT_CONSTRUCTOR.newInstance(conf, jobId);
+    } catch (InstantiationException e) {
+      throw new IllegalArgumentException("Can't instantiate JobContext", e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException("Can't instantiate JobContext", e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalArgumentException("Can't instantiate JobContext", e);
+    }
+  }
+
+  /**
+   * Creates TaskAttempContext from a JobConf and jobId using the correct
+   * constructor for based on Hadoop version.
+   */
+  public static TaskAttemptContext newTaskAttemptContext(
+      Configuration conf, TaskAttemptID taskAttemptId) {
+    try {
+      return (TaskAttemptContext)
+          TASK_CONTEXT_CONSTRUCTOR.newInstance(conf, taskAttemptId);
+    } catch (InstantiationException e) {
+      throw new IllegalArgumentException("Can't instantiate TaskAttemptContext", e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException("Can't instantiate TaskAttemptContext", e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalArgumentException("Can't instantiate TaskAttemptContext", e);
+    }
+  }
+
+  /**
+   * @return with Hadoop 2 : <code>new GenericCounter(args)</code>,<br>
+   *         with Hadoop 1 : <code>new Counter(args)</code>
+   */
+  public static Counter newGenericCounter(String name, String displayName, long value) {
+    try {
+      return (Counter)
+          GENERIC_COUNTER_CONSTRUCTOR.newInstance(name, displayName, value);
+    } catch (InstantiationException e) {
+      throw new IllegalArgumentException("Can't instantiate Counter", e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException("Can't instantiate Counter", e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalArgumentException("Can't instantiate Counter", e);
+    }
+  }
+
+  /**
+   * Invoke getConfiguration() method on JobContext. Works with both
+   * Hadoop 1 and 2.
+   */
+  public static Configuration getConfiguration(JobContext context) {
+    try {
+      return (Configuration) GET_CONFIGURATION_METHOD.invoke(context);
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException("Can't invoke method", e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalArgumentException("Can't invoke method", e);
+    }
+  }
+}
\ No newline at end of file
diff --git a/parquet-hadoop/src/test/java/parquet/format/converter/TestParquetMetadataConverter.java b/parquet-hadoop/src/test/java/parquet/format/converter/TestParquetMetadataConverter.java
index 78ab4db..33e2da5 100644
--- a/parquet-hadoop/src/test/java/parquet/format/converter/TestParquetMetadataConverter.java
+++ b/parquet-hadoop/src/test/java/parquet/format/converter/TestParquetMetadataConverter.java
@@ -15,7 +15,9 @@
  */
 package parquet.format.converter;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static parquet.format.Util.readPageHeader;
+import static parquet.format.Util.writePageHeader;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -31,7 +33,6 @@
 import parquet.format.PageType;
 import parquet.format.SchemaElement;
 import parquet.format.Type;
-import parquet.format.converter.ParquetMetadataConverter;
 import parquet.schema.MessageType;
 import parquet.schema.PrimitiveType.PrimitiveTypeName;
 import parquet.schema.Type.Repetition;
@@ -40,14 +41,13 @@
 
   @Test
   public void testPageHeader() throws IOException {
-    ParquetMetadataConverter parquetMetadataConverter = new ParquetMetadataConverter();
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     PageType type = PageType.DATA_PAGE;
     int compSize = 10;
     int uncSize = 20;
     PageHeader pageHeader = new PageHeader(type, uncSize, compSize);
-    parquetMetadataConverter.writePageHeader(pageHeader, out);
-    PageHeader readPageHeader = parquetMetadataConverter.readPageHeader(new ByteArrayInputStream(out.toByteArray()));
+    writePageHeader(pageHeader, out);
+    PageHeader readPageHeader = readPageHeader(new ByteArrayInputStream(out.toByteArray()));
     assertEquals(pageHeader, readPageHeader);
   }
 
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java b/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java
index 44d5eed..deb66cc 100644
--- a/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java
@@ -26,7 +26,6 @@
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.InputSplit;
 import org.junit.Test;
 
 import parquet.column.Encoding;
@@ -36,6 +35,7 @@
 import parquet.hadoop.metadata.CompressionCodecName;
 import parquet.hadoop.metadata.FileMetaData;
 import parquet.schema.MessageType;
+import parquet.schema.MessageTypeParser;
 import parquet.schema.PrimitiveType.PrimitiveTypeName;
 
 public class TestInputFormat {
@@ -51,15 +51,19 @@
         new BlockLocation(new String[0], new String[] { "foo1.datanode", "bar1.datanode"}, 50, 50)
     };
     FileStatus fileStatus = new FileStatus(100, false, 2, 50, 0, new Path("hdfs://foo.namenode:1234/bar"));
-    FileMetaData fileMetaData = new FileMetaData(new MessageType("foo"), new HashMap<String, String>());
-    List<InputSplit> splits = ParquetInputFormat.generateSplits(blocks, hdfsBlocks, fileStatus, fileMetaData, ReadSupport.class, "");
+    MessageType schema = MessageTypeParser.parseMessageType("message doc { required binary foo; }");
+    FileMetaData fileMetaData = new FileMetaData(schema, new HashMap<String, String>(), "parquet-mr");
+    @SuppressWarnings("serial")
+    List<ParquetInputSplit> splits = ParquetInputFormat.generateSplits(
+        blocks, hdfsBlocks, fileStatus, fileMetaData, ReadSupport.class, schema.toString(), new HashMap<String, String>() {{put("specific", "foo");}});
     assertEquals(splits.toString().replaceAll("([{])", "$0\n").replaceAll("([}])", "\n$0"), 2, splits.size());
     for (int i = 0; i < splits.size(); i++) {
-      ParquetInputSplit parquetInputSplit = (ParquetInputSplit)splits.get(i);
+      ParquetInputSplit parquetInputSplit = splits.get(i);
       assertEquals(5, parquetInputSplit.getBlocks().size());
       assertEquals(2, parquetInputSplit.getLocations().length);
       assertEquals("[foo" + i + ".datanode, bar" + i + ".datanode]", Arrays.toString(parquetInputSplit.getLocations()));
-      assertEquals(50, parquetInputSplit.getLength());
+      assertEquals(10, parquetInputSplit.getLength());
+      assertEquals("foo", parquetInputSplit.getReadSupportMetadata().get("specific"));
     }
   }
 
@@ -67,6 +71,7 @@
     BlockMetaData blockMetaData = new BlockMetaData();
     ColumnChunkMetaData column = new ColumnChunkMetaData(new String[] {"foo"}, PrimitiveTypeName.BINARY, CompressionCodecName.GZIP, Arrays.asList(Encoding.PLAIN));
     column.setFirstDataPageOffset(start);
+    column.setTotalSize(2);
     blockMetaData.addColumn(column);
     return blockMetaData;
   }
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java b/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
index d3669c8..2c0ac71 100644
--- a/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
@@ -74,14 +74,18 @@
     w.start();
     w.startBlock(3);
     w.startColumn(c1, 5, codec);
+    long c1Starts = w.getPos();
     w.writeDataPage(2, 4, BytesInput.from(bytes1), BIT_PACKED, BIT_PACKED, PLAIN);
     w.writeDataPage(3, 4, BytesInput.from(bytes1), BIT_PACKED, BIT_PACKED, PLAIN);
     w.endColumn();
+    long c1Ends = w.getPos();
     w.startColumn(c2, 6, codec);
+    long c2Starts = w.getPos();
     w.writeDataPage(2, 4, BytesInput.from(bytes2), BIT_PACKED, BIT_PACKED, PLAIN);
     w.writeDataPage(3, 4, BytesInput.from(bytes2), BIT_PACKED, BIT_PACKED, PLAIN);
     w.writeDataPage(1, 4, BytesInput.from(bytes2), BIT_PACKED, BIT_PACKED, PLAIN);
     w.endColumn();
+    long c2Ends = w.getPos();
     w.endBlock();
     w.startBlock(4);
     w.startColumn(c1, 7, codec);
@@ -95,6 +99,9 @@
 
     ParquetMetadata readFooter = ParquetFileReader.readFooter(configuration, path);
     assertEquals("footer: "+readFooter, 2, readFooter.getBlocks().size());
+    assertEquals(c1Ends - c1Starts, readFooter.getBlocks().get(0).getColumns().get(0).getTotalSize());
+    assertEquals(c2Ends - c2Starts, readFooter.getBlocks().get(0).getColumns().get(1).getTotalSize());
+
 
     { // read first block of col #1
       ParquetFileReader r = new ParquetFileReader(configuration, path, Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(schema.getColumnDescription(path1)));
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/TestSnappyCodec.java b/parquet-hadoop/src/test/java/parquet/hadoop/TestSnappyCodec.java
new file mode 100644
index 0000000..09c9d68
--- /dev/null
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestSnappyCodec.java
@@ -0,0 +1,82 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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;
+
+import java.io.IOException;
+
+import org.junit.Test;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import parquet.hadoop.codec.SnappyCompressor;
+import parquet.hadoop.codec.SnappyDecompressor;
+
+public class TestSnappyCodec {
+  @Test
+  public void TestSnappy() throws IOException {
+    // Reuse the snappy objects between test cases
+    SnappyCompressor compressor = new SnappyCompressor();
+    SnappyDecompressor decompressor = new SnappyDecompressor();
+
+    TestSnappy(compressor, decompressor, "");    
+    TestSnappy(compressor, decompressor, "FooBar");    
+    TestSnappy(compressor, decompressor, "FooBar1", "FooBar2");    
+    TestSnappy(compressor, decompressor, "FooBar");
+    TestSnappy(compressor, decompressor, "a", "blahblahblah", "abcdef");    
+    TestSnappy(compressor, decompressor, "");
+    TestSnappy(compressor, decompressor, "FooBar");
+  }
+
+  private void TestSnappy(SnappyCompressor compressor, SnappyDecompressor decompressor, 
+      String... strings) throws IOException {
+    compressor.reset();
+    decompressor.reset();
+
+    int uncompressedSize = 0;
+    for (String s: strings) {
+      uncompressedSize += s.length();
+    }
+    byte[] uncompressedData = new byte[uncompressedSize];
+    int len = 0;
+    for (String s: strings) {
+      byte[] tmp = s.getBytes();
+      System.arraycopy(tmp, 0, uncompressedData, len, s.length());
+      len += s.length();
+    }
+
+    assert(compressor.needsInput());
+    compressor.setInput(uncompressedData, 0, len);
+    assert(compressor.needsInput());
+    compressor.finish();
+    assert(!compressor.needsInput());
+    assert(!compressor.finished() || uncompressedSize == 0);
+    byte[] compressedData = new byte[1000];
+
+    int compressedSize = compressor.compress(compressedData, 0, 1000);
+    assert(compressor.finished());
+
+    assert(!decompressor.finished());
+    assert(decompressor.needsInput());
+    decompressor.setInput(compressedData, 0, compressedSize);
+    assert(!decompressor.finished());
+    byte[] decompressedData = new byte[uncompressedSize];
+    int decompressedSize = decompressor.decompress(decompressedData, 0, uncompressedSize);
+    assert(decompressor.finished());
+
+    assertEquals(uncompressedSize, decompressedSize);
+    assertArrayEquals(uncompressedData, decompressedData);
+  }
+}
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java b/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java
index c7c75d5..168c022 100644
--- a/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java
@@ -23,10 +23,8 @@
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
-import java.util.List;
 
 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.io.LongWritable;
@@ -40,10 +38,8 @@
 import parquet.Log;
 import parquet.example.data.Group;
 import parquet.example.data.simple.SimpleGroupFactory;
-import parquet.hadoop.Footer;
-import parquet.hadoop.ParquetFileReader;
-import parquet.hadoop.ParquetFileWriter;
 import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.hadoop.util.ContextUtil;
 import parquet.schema.MessageTypeParser;
 
 public class TestInputOutputFormat {
@@ -52,7 +48,7 @@
   public static class MyMapper extends Mapper<LongWritable, Text, Void, Group> {
     private SimpleGroupFactory factory;
     protected void setup(org.apache.hadoop.mapreduce.Mapper<LongWritable,Text,Void,Group>.Context context) throws java.io.IOException ,InterruptedException {
-      factory = new SimpleGroupFactory(GroupWriteSupport.getSchema(context.getConfiguration()));
+      factory = new SimpleGroupFactory(GroupWriteSupport.getSchema(ContextUtil.getConfiguration(context)));
     };
     protected void map(LongWritable key, Text value, Mapper<LongWritable,Text,Void,Group>.Context context) throws java.io.IOException ,InterruptedException {
       Group group = factory.newGroup()
@@ -66,11 +62,9 @@
     protected void map(Void key, Group value, Mapper<Void,Group,LongWritable,Text>.Context context) throws IOException ,InterruptedException {
       context.write(new LongWritable(value.getInteger("line", 0)), new Text(value.getString("content", 0)));
     }
-
   }
 
-  @Test
-  public void testReadWrite() throws IOException, ClassNotFoundException, InterruptedException {
+  private void testReadWrite(CompressionCodecName codec) throws IOException, ClassNotFoundException, InterruptedException {
     final Configuration conf = new Configuration();
     final Path inputPath = new Path("src/test/java/parquet/hadoop/example/TestInputOutputFormat.java");
     final Path parquetPath = new Path("target/test/example/TestInputOutputFormat/parquet");
@@ -83,7 +77,7 @@
       TextInputFormat.addInputPath(job, inputPath);
       job.setInputFormatClass(TextInputFormat.class);
       job.setNumReduceTasks(0);
-      ExampleOutputFormat.setCompression(job, CompressionCodecName.GZIP);
+      ExampleOutputFormat.setCompression(job, codec);
       ExampleOutputFormat.setOutputPath(job, parquetPath);
       job.setOutputFormatClass(ExampleOutputFormat.class);
       job.setMapperClass(TestInputOutputFormat.MyMapper.class);
@@ -124,6 +118,13 @@
     in.close();
     out.close();
   }
+  @Test
+  public void testReadWrite() throws IOException, ClassNotFoundException, InterruptedException {
+    // TODO: Lzo requires additional external setup steps so leave it out for now
+    testReadWrite(CompressionCodecName.GZIP);
+    testReadWrite(CompressionCodecName.UNCOMPRESSED);
+    testReadWrite(CompressionCodecName.SNAPPY);
+  }
 
   private void waitForJob(Job job) throws InterruptedException, IOException {
     while (!job.isComplete()) {
diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml
index 7a60e15..4fde94c 100644
--- a/parquet-pig/pom.xml
+++ b/parquet-pig/pom.xml
@@ -41,8 +41,18 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-core</artifactId>
-      <version>1.0.0</version>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <version>${jackson.version}</version>
     </dependency>
     <dependency>
       <groupId>log4j</groupId>
@@ -81,6 +91,14 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
         <executions>
           <execution>
             <goals>
@@ -89,6 +107,33 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.0</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <artifactSet>
+                <includes>
+                  <include>org.codehaus.jackson:jackson-mapper-asl</include>
+                  <include>org.codehaus.jackson:jackson-core-asl</include>
+                </includes>
+              </artifactSet>
+              <relocations>
+                <relocation>
+                  <pattern>org.codehaus.jackson</pattern>
+                  <shadedPattern>parquet.org.codehaus.jackson</shadedPattern>
+                </relocation>
+              </relocations>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 
diff --git a/parquet-pig/src/main/java/parquet/pig/ParquetLoader.java b/parquet-pig/src/main/java/parquet/pig/ParquetLoader.java
index 9731a44..a193472 100644
--- a/parquet-pig/src/main/java/parquet/pig/ParquetLoader.java
+++ b/parquet-pig/src/main/java/parquet/pig/ParquetLoader.java
@@ -18,8 +18,6 @@
 import static parquet.pig.TupleReadSupport.PARQUET_PIG_REQUESTED_SCHEMA;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
@@ -37,9 +35,9 @@
 import org.apache.pig.parser.ParserException;
 
 import parquet.Log;
-import parquet.hadoop.Footer;
 import parquet.hadoop.ParquetInputFormat;
 import parquet.hadoop.metadata.FileMetaData;
+import parquet.hadoop.util.ContextUtil;
 import parquet.io.ParquetDecodingException;
 
 /**
@@ -80,7 +78,7 @@
     LOG.debug("LoadFunc.setLocation(" + location + ", " + job + ")");
     setInput(location, job);
     if (requestedSchema != null) {
-      job.getConfiguration().set(PARQUET_PIG_REQUESTED_SCHEMA, requestedSchema);
+      ContextUtil.getConfiguration(job).set(PARQUET_PIG_REQUESTED_SCHEMA, requestedSchema);
     }
   }
 
diff --git a/parquet-pig/src/main/java/parquet/pig/ParquetStorer.java b/parquet-pig/src/main/java/parquet/pig/ParquetStorer.java
index f4b2d6c..247f308 100644
--- a/parquet-pig/src/main/java/parquet/pig/ParquetStorer.java
+++ b/parquet-pig/src/main/java/parquet/pig/ParquetStorer.java
@@ -65,7 +65,11 @@
 
   private Schema getSchema() {
     try {
-      return Utils.getSchemaFromString(getProperties().getProperty(SCHEMA));
+      final String schemaString = getProperties().getProperty(SCHEMA);
+      if (schemaString == null) {
+        throw new ParquetEncodingException("Can not store relation in Parquet as the schema is unknown");
+      }
+      return Utils.getSchemaFromString(schemaString);
     } catch (ParserException e) {
       throw new ParquetEncodingException("can not get schema from context", e);
     }
diff --git a/parquet-pig/src/main/java/parquet/pig/convert/TupleConverter.java b/parquet-pig/src/main/java/parquet/pig/convert/TupleConverter.java
index c9ea639..cfede6d 100644
--- a/parquet-pig/src/main/java/parquet/pig/convert/TupleConverter.java
+++ b/parquet-pig/src/main/java/parquet/pig/convert/TupleConverter.java
@@ -28,6 +28,7 @@
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
 
+import parquet.column.Dictionary;
 import parquet.io.ParquetDecodingException;
 import parquet.io.api.Binary;
 import parquet.io.api.Converter;
@@ -37,7 +38,6 @@
 import parquet.schema.GroupType;
 import parquet.schema.OriginalType;
 import parquet.schema.PrimitiveType;
-import parquet.schema.PrimitiveType.PrimitiveTypeName;
 import parquet.schema.Type;
 import parquet.schema.Type.Repetition;
 
@@ -186,7 +186,8 @@
   }
 
   /**
-   * handle string values
+   * handle string values.
+   * In case of dictionary encoding, the strings will be decoded only once.
    * @author Julien Le Dem
    *
    */
@@ -194,6 +195,8 @@
 
     private final ParentValueContainer parent;
 
+    private String[] dict;
+
     public FieldStringConverter(ParentValueContainer parent) {
       this.parent = parent;
     }
@@ -203,6 +206,23 @@
       parent.add(value.toStringUsingUTF8());
     }
 
+    @Override
+    public boolean hasDictionarySupport() {
+      return true;
+    }
+
+    @Override
+    public void setDictionary(Dictionary dictionary) {
+      dict = new String[dictionary.getMaxId() + 1];
+      for (int i = 0; i <= dictionary.getMaxId(); i++) {
+        dict[i] = dictionary.decodeToBinary(i).toStringUsingUTF8();
+      }
+    }
+
+    @Override
+    public void addValueFromDictionary(int dictionaryId) {
+      parent.add(dict[dictionaryId]);
+    }
   }
 
   /**
diff --git a/parquet-pig/src/main/java/parquet/pig/summary/ValueStat.java b/parquet-pig/src/main/java/parquet/pig/summary/ValueStat.java
index 4c2fb8a..2f3db3d 100644
--- a/parquet-pig/src/main/java/parquet/pig/summary/ValueStat.java
+++ b/parquet-pig/src/main/java/parquet/pig/summary/ValueStat.java
@@ -18,8 +18,8 @@
 public class ValueStat {
 
   private double total;
-  private double min = Double.MAX_VALUE;
-  private double max = Double.MIN_VALUE;
+  private double min = Double.POSITIVE_INFINITY;
+  private double max = Double.NEGATIVE_INFINITY;
 
   public void add(double v) {
     total += v;
diff --git a/parquet-pig/src/test/java/parquet/pig/GenerateIntTestFile.java b/parquet-pig/src/test/java/parquet/pig/GenerateIntTestFile.java
index e4bf9ae..dc3a9c2 100644
--- a/parquet-pig/src/test/java/parquet/pig/GenerateIntTestFile.java
+++ b/parquet-pig/src/test/java/parquet/pig/GenerateIntTestFile.java
@@ -15,22 +15,17 @@
  */
 package parquet.pig;
 
-import static parquet.column.Encoding.BIT_PACKED;
-import static parquet.column.Encoding.PLAIN;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
 import parquet.Log;
 import parquet.bytes.BytesInput;
 import parquet.column.ColumnDescriptor;
-import parquet.column.Encoding;
 import parquet.column.impl.ColumnWriteStoreImpl;
 import parquet.column.page.Page;
 import parquet.column.page.PageReadStore;
@@ -64,8 +59,7 @@
       MessageType schema = new MessageType("int_test_file", new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.INT32, "int_col"));
 
       MemPageStore pageStore = new MemPageStore();
-      ColumnWriteStoreImpl store = new ColumnWriteStoreImpl(pageStore, 8*1024);
-      //
+      ColumnWriteStoreImpl store = new ColumnWriteStoreImpl(pageStore, 8*1024, 8*1024, false);
       MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
 
       RecordConsumer recordWriter = columnIO.getRecordWriter(store);
diff --git a/parquet-pig/src/test/java/parquet/pig/GenerateTPCH.java b/parquet-pig/src/test/java/parquet/pig/GenerateTPCH.java
index c181cb4..236fa06 100644
--- a/parquet-pig/src/test/java/parquet/pig/GenerateTPCH.java
+++ b/parquet-pig/src/test/java/parquet/pig/GenerateTPCH.java
@@ -61,9 +61,8 @@
         );
 
     MemPageStore pageStore = new MemPageStore();
-    ColumnWriteStoreImpl store = new ColumnWriteStoreImpl(pageStore, 8*1024);
-    //
-    MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
+    ColumnWriteStoreImpl store = new ColumnWriteStoreImpl(pageStore, 20*1024, 1*1024, false);
+    MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
 
     RecordConsumer recordWriter = columnIO.getRecordWriter(store);
 
@@ -82,7 +81,8 @@
       ++ recordCount;
     }
     store.flush();
-
+    System.out.printf("mem size %,d, maxColSize %,d, allocated %,d\n", store.memSize(), store.maxColMemSize(), store.allocatedSize());
+    System.out.println(store.memUsageString());
     writeToFile(testFile, configuration, schema, pageStore, recordCount);
 
     try {
@@ -94,8 +94,8 @@
   }
 
   private static void writeField(RecordConsumer recordWriter, int index, String name, Object value) {
-    recordWriter.startField(name, index);
     if (value != null) {
+      recordWriter.startField(name, index);
       if (value instanceof Integer) {
         recordWriter.addInteger((Integer)value);
       } else if (value instanceof String) {
@@ -105,7 +105,7 @@
       } else {
         throw new IllegalArgumentException(value.getClass().getName() + " not supported");
       }
+      recordWriter.endField(name, index);
     }
-    recordWriter.endField(name, index);
   }
 }
diff --git a/parquet-pig/src/test/java/parquet/pig/PerfTest2.java b/parquet-pig/src/test/java/parquet/pig/PerfTest2.java
index cf1ad2c..e95e79d 100644
--- a/parquet-pig/src/test/java/parquet/pig/PerfTest2.java
+++ b/parquet-pig/src/test/java/parquet/pig/PerfTest2.java
@@ -46,6 +46,7 @@
 import org.apache.pig.parser.ParserException;
 
 import parquet.Log;
+import parquet.hadoop.util.ContextUtil;
 import parquet.pig.ParquetLoader;
 import parquet.pig.ParquetStorer;
 
@@ -105,8 +106,8 @@
       storer.checkSchema(new ResourceSchema(Utils.getSchemaFromString(schema)));
       @SuppressWarnings("unchecked") // that's how the base class is defined
       OutputFormat<Void, Tuple> outputFormat = storer.getOutputFormat();
-      // it's job.getConfiguration() and not just conf !
-      JobContext jobContext = new JobContext(job.getConfiguration(), new JobID("jt", jobid ++));
+      // it's ContextUtil.getConfiguration(job) and not just conf !
+      JobContext jobContext = new JobContext(ContextUtil.getConfiguration(job), new JobID("jt", jobid ++));
       outputFormat.checkOutputSpecs(jobContext);
       if (schema != null) {
         ResourceSchema resourceSchema = new ResourceSchema(Utils.getSchemaFromString(schema));
@@ -115,7 +116,7 @@
           ((StoreMetadata)storer).storeSchema(resourceSchema, absPath, job);
         }
       }
-      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID("jt", jobid, true, 1, 0));
+      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(ContextUtil.getConfiguration(job), new TaskAttemptID("jt", jobid, true, 1, 0));
       RecordWriter<Void, Tuple> recordWriter = outputFormat.getRecordWriter(taskAttemptContext);
       storer.prepareToWrite(recordWriter);
 
@@ -160,12 +161,12 @@
     loadFunc.setLocation(absPath, job);
     @SuppressWarnings("unchecked") // that's how the base class is defined
     InputFormat<Void, Tuple> inputFormat = loadFunc.getInputFormat();
-    JobContext jobContext = new JobContext(job.getConfiguration(), new JobID("jt", loadjobId));
+    JobContext jobContext = new JobContext(ContextUtil.getConfiguration(job), new JobID("jt", loadjobId));
     List<InputSplit> splits = inputFormat.getSplits(jobContext);
     int i = 0;
     int taskid = 0;
     for (InputSplit split : splits) {
-      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID("jt", loadjobId, true, taskid++, 0));
+      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(ContextUtil.getConfiguration(job), new TaskAttemptID("jt", loadjobId, true, taskid++, 0));
       RecordReader<Void, Tuple> recordReader = inputFormat.createRecordReader(split, taskAttemptContext);
       loadFunc.prepareToRead(recordReader, null);
       recordReader.initialize(split, taskAttemptContext);
diff --git a/parquet-pig/src/test/java/parquet/pig/TupleConsumerPerfTest.java b/parquet-pig/src/test/java/parquet/pig/TupleConsumerPerfTest.java
index efc700f..e29f095 100644
--- a/parquet-pig/src/test/java/parquet/pig/TupleConsumerPerfTest.java
+++ b/parquet-pig/src/test/java/parquet/pig/TupleConsumerPerfTest.java
@@ -56,7 +56,7 @@
     MessageType schema = pigSchemaConverter.convert(Utils.getSchemaFromString(pigSchema));
 
     MemPageStore memPageStore = new MemPageStore();
-    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 50*1024*1024);
+    ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 50*1024*1024, 50*1024*1024, false);
     write(columns, schema, pigSchema);
     columns.flush();
     read(memPageStore, pigSchema, pigSchemaProjected, pigSchemaNoString);
diff --git a/parquet-pig/src/test/java/parquet/pig/summary/TestSummary.java b/parquet-pig/src/test/java/parquet/pig/summary/TestSummary.java
index e936785..6e7a990 100644
--- a/parquet-pig/src/test/java/parquet/pig/summary/TestSummary.java
+++ b/parquet-pig/src/test/java/parquet/pig/summary/TestSummary.java
@@ -140,5 +140,23 @@
     System.out.println(s);
   }
 
+  @Test
+  public void testMaxIsZero() throws Exception {
+    PigServer pigServer = new PigServer(ExecType.LOCAL);
+    Data data = Storage.resetData(pigServer);
+
+    List<Tuple> list = new ArrayList<Tuple>();
+    for (int i = 0; i < 10; i++) {
+      list.add(t("a", i - 9));
+    }
+    
+    data.set("in", "a:chararray, b:int", list);
+    pigServer.registerQuery("A = LOAD 'in' USING mock.Storage();");
+    pigServer.registerQuery("B = FOREACH (GROUP A ALL) GENERATE " + Summary.class.getName() + "(A);");
+    pigServer.registerQuery("STORE B INTO 'out' USING mock.Storage();");
+    TupleSummaryData s = SummaryData.fromJSON((String) data.get("out").get(0).get(0), TupleSummaryData.class);
+    System.out.println(s);	  
+    assertEquals(0, s.getFields().get(1).getNumber().getValue().getMax(), 0);
+  }
 
 }
diff --git a/parquet-scrooge/pom.xml b/parquet-scrooge/pom.xml
new file mode 100644
index 0000000..960fbb5
--- /dev/null
+++ b/parquet-scrooge/pom.xml
@@ -0,0 +1,110 @@
+<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.0.0-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-scrooge</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Parquet Scrooge</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <repositories>
+    <repository>
+     <id>conjars.org</id>
+      <url>http://conjars.org/repo</url>
+    </repository>
+  </repositories>
+  
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</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.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <version>1.2.17</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-thrift</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-cascading</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>scalding_2.9.2</artifactId>
+      <version>0.8.3</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>scrooge-runtime</artifactId>
+      <version>3.0.9</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-thrift</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java b/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java
new file mode 100644
index 0000000..b2199fe
--- /dev/null
+++ b/parquet-scrooge/src/main/java/parquet/scrooge/ParquetScroogeScheme.java
@@ -0,0 +1,74 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.scrooge;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
+
+import parquet.cascading.ParquetValueScheme;
+import parquet.hadoop.ParquetInputFormat;
+import parquet.hadoop.mapred.DeprecatedParquetInputFormat;
+import parquet.hadoop.thrift.ParquetThriftInputFormat;
+import parquet.hadoop.thrift.ThriftReadSupport;
+import cascading.flow.FlowProcess;
+import cascading.scheme.SinkCall;
+import cascading.tap.Tap;
+
+import com.twitter.scrooge.ThriftStruct;
+
+public class ParquetScroogeScheme<T extends ThriftStruct> extends ParquetValueScheme<T> {
+
+  private static final long serialVersionUID = -8332274507341448397L;
+  private final Class<T> klass;
+
+  public ParquetScroogeScheme(Class<T> klass) {
+    this.klass = klass;
+  }
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void sinkConfInit(FlowProcess<JobConf> arg0,
+      Tap<JobConf, RecordReader, OutputCollector> arg1, JobConf arg2) {
+    throw new UnsupportedOperationException("ParquetScroogeScheme does not support Sinks");
+  }
+
+  /**
+   * TODO: currently we cannot write Parquet files from Scrooge objects.
+   */
+  @Override
+  public boolean isSink() { return false; }
+
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void sourceConfInit(FlowProcess<JobConf> fp,
+      Tap<JobConf, RecordReader, OutputCollector> tap, JobConf jobConf) {
+    jobConf.setInputFormat(DeprecatedParquetInputFormat.class);
+    ParquetInputFormat.setReadSupportClass(jobConf, ThriftReadSupport.class);
+    ThriftReadSupport.setRecordConverterClass(jobConf, ScroogeRecordConverter.class);
+    ParquetThriftInputFormat.<T>setThriftClass(jobConf, klass);
+  }
+
+  @Override
+  public void sink(FlowProcess<JobConf> arg0, SinkCall<Object[], OutputCollector> arg1)
+      throws IOException {
+    throw new UnsupportedOperationException("ParquetScroogeScheme does not support Sinks");
+  }
+}
diff --git a/parquet-scrooge/src/main/java/parquet/scrooge/ScroogeRecordConverter.java b/parquet-scrooge/src/main/java/parquet/scrooge/ScroogeRecordConverter.java
new file mode 100644
index 0000000..bfd6c51
--- /dev/null
+++ b/parquet-scrooge/src/main/java/parquet/scrooge/ScroogeRecordConverter.java
@@ -0,0 +1,54 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.scrooge;
+
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TProtocol;
+
+import com.twitter.scrooge.ThriftStruct;
+import com.twitter.scrooge.ThriftStructCodec;
+
+import parquet.schema.MessageType;
+import parquet.thrift.ThriftReader;
+import parquet.thrift.ThriftRecordConverter;
+import parquet.thrift.struct.ThriftType.StructType;
+
+public class ScroogeRecordConverter<T extends ThriftStruct> extends ThriftRecordConverter<T> {
+
+
+  public ScroogeRecordConverter(final Class<T> thriftClass, MessageType parquetSchema, StructType thriftType) {
+    super(new ThriftReader<T>() {
+      @SuppressWarnings("unchecked")
+      ThriftStructCodec<T> codec = (ThriftStructCodec<T>) getCodec(thriftClass);
+      @Override
+      public T readOneRecord(TProtocol protocol) throws TException {
+          return codec.decode(protocol);
+      }
+    }, thriftClass.getSimpleName(), parquetSchema, thriftType);
+  }
+
+  private static ThriftStructCodec<?> getCodec(Class<?> klass) {
+    Class<?> companionClass;
+    try {
+      companionClass = Class.forName(klass.getName() + "$");
+      Object companionObject = companionClass.getField("MODULE$").get(null);
+      return (ThriftStructCodec<?>) companionObject;
+    } catch (Exception t) {
+      if (t instanceof InterruptedException) Thread.currentThread().interrupt();
+      throw new RuntimeException("Unable to create ThriftStructCodec", t);
+    }
+  }
+}
diff --git a/parquet-test-hadoop2/pom.xml b/parquet-test-hadoop2/pom.xml
new file mode 100644
index 0000000..2dcd2b2
--- /dev/null
+++ b/parquet-test-hadoop2/pom.xml
@@ -0,0 +1,74 @@
+<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.0.0-SNAPSHOT</version>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-test-hadoop2</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Parquet Hadoop 2 tests</name>
+  <url>https://github.com/Parquet/parquet-mr</url>
+
+  <properties>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <version>2.0.3-alpha</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+
+</project>
diff --git a/parquet-test-hadoop2/src/test/java/parquet/hadoop2/TestInputOutputFormat.java b/parquet-test-hadoop2/src/test/java/parquet/hadoop2/TestInputOutputFormat.java
new file mode 100644
index 0000000..002044d
--- /dev/null
+++ b/parquet-test-hadoop2/src/test/java/parquet/hadoop2/TestInputOutputFormat.java
@@ -0,0 +1,139 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.hadoop2;
+
+import static java.lang.Thread.sleep;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.junit.Test;
+
+import parquet.Log;
+import parquet.example.data.Group;
+import parquet.example.data.simple.SimpleGroupFactory;
+import parquet.hadoop.example.ExampleInputFormat;
+import parquet.hadoop.example.ExampleOutputFormat;
+import parquet.hadoop.example.GroupWriteSupport;
+import parquet.hadoop.metadata.CompressionCodecName;
+import parquet.schema.MessageTypeParser;
+
+public class TestInputOutputFormat {
+  private static final Log LOG = Log.getLog(TestInputOutputFormat.class);
+
+  public static class MyMapper extends Mapper<LongWritable, Text, Void, Group> {
+    private SimpleGroupFactory factory;
+    protected void setup(org.apache.hadoop.mapreduce.Mapper<LongWritable,Text,Void,Group>.Context context) throws java.io.IOException ,InterruptedException {
+      factory = new SimpleGroupFactory(GroupWriteSupport.getSchema(context.getConfiguration()));
+    };
+    protected void map(LongWritable key, Text value, Mapper<LongWritable,Text,Void,Group>.Context context) throws java.io.IOException ,InterruptedException {
+      Group group = factory.newGroup()
+          .append("line", (int)key.get())
+          .append("content", value.toString());
+      context.write(null, group);
+    }
+  }
+
+  public static class MyMapper2 extends Mapper<Void, Group, LongWritable, Text> {
+    protected void map(Void key, Group value, Mapper<Void,Group,LongWritable,Text>.Context context) throws IOException ,InterruptedException {
+      context.write(new LongWritable(value.getInteger("line", 0)), new Text(value.getString("content", 0)));
+    }
+
+  }
+
+  @Test
+  public void testReadWrite() throws IOException, ClassNotFoundException, InterruptedException {
+    final Configuration conf = new Configuration();
+    final Path inputPath = new Path("src/test/java/parquet/hadoop2/TestInputOutputFormat.java");
+    final Path parquetPath = new Path("target/test/hadoop2/example/TestInputOutputFormat/parquet");
+    final Path outputPath = new Path("target/test/hadoop2/example/TestInputOutputFormat/out");
+    final FileSystem fileSystem = parquetPath.getFileSystem(conf);
+    fileSystem.delete(parquetPath, true);
+    fileSystem.delete(outputPath, true);
+    {
+      final Job job = new Job(conf, "write");
+      TextInputFormat.addInputPath(job, inputPath);
+      job.setInputFormatClass(TextInputFormat.class);
+      job.setNumReduceTasks(0);
+      ExampleOutputFormat.setCompression(job, CompressionCodecName.GZIP);
+      ExampleOutputFormat.setOutputPath(job, parquetPath);
+      job.setOutputFormatClass(ExampleOutputFormat.class);
+      job.setMapperClass(TestInputOutputFormat.MyMapper.class);
+      ExampleOutputFormat.setSchema(
+          job,
+          MessageTypeParser.parseMessageType(
+              "message example {\n" +
+              "required int32 line;\n" +
+              "required binary content;\n" +
+              "}"));
+      job.submit();
+      waitForJob(job);
+    }
+    {
+      final Job job = new Job(conf, "read");
+      job.setInputFormatClass(ExampleInputFormat.class);
+      ExampleInputFormat.setInputPaths(job, parquetPath);
+      job.setOutputFormatClass(TextOutputFormat.class);
+      TextOutputFormat.setOutputPath(job, outputPath);
+      job.setMapperClass(TestInputOutputFormat.MyMapper2.class);
+      job.setNumReduceTasks(0);
+      job.submit();
+      waitForJob(job);
+    }
+
+    final BufferedReader in = new BufferedReader(new FileReader(new File(inputPath.toString())));
+    final BufferedReader out = new BufferedReader(new FileReader(new File(outputPath.toString(), "part-m-00000")));
+    String lineIn;
+    String lineOut = null;
+    int lineNumber = 0;
+    while ((lineIn = in.readLine()) != null && (lineOut = out.readLine()) != null) {
+      ++ lineNumber;
+      lineOut = lineOut.substring(lineOut.indexOf("\t") + 1);
+      assertEquals("line " + lineNumber, lineIn, lineOut);
+    }
+    assertNull("line " + lineNumber, lineIn);
+    assertNull("line " + lineNumber, out.readLine());
+    assertTrue(lineNumber > 0);
+    in.close();
+    out.close();
+  }
+
+  private void waitForJob(Job job) throws InterruptedException, IOException {
+    while (!job.isComplete()) {
+      LOG.debug("waiting for job " + job.getJobName());
+      sleep(100);
+    }
+    LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE"));
+    if (!job.isSuccessful()) {
+      throw new RuntimeException("job failed " + job.getJobName());
+    }
+  }
+
+}
diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml
index e813102..124eefd 100644
--- a/parquet-thrift/pom.xml
+++ b/parquet-thrift/pom.xml
@@ -32,8 +32,8 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-core</artifactId>
-      <version>1.0.0</version>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
     </dependency>
     <dependency>
       <groupId>com.twitter.elephantbird</groupId>
@@ -48,13 +48,13 @@
     <dependency>
       <groupId>org.codehaus.jackson</groupId>
       <artifactId>jackson-mapper-asl</artifactId>
-      <version>1.9.11</version>
+      <version>${jackson.version}</version>
     </dependency>
-<!--    <dependency>
+    <dependency>
       <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-annotations</artifactId>
-      <version>1.9.11</version>
-    </dependency> -->
+      <artifactId>jackson-core-asl</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
     <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
@@ -73,4 +73,54 @@
       <version>${project.version}</version>
     </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <configuration>
+          <archive>
+            <manifestEntries>
+              <git-SHA-1>${buildNumber}</git-SHA-1>
+            </manifestEntries>
+          </archive>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.0</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <artifactSet>
+                <includes>
+                  <include>org.codehaus.jackson:jackson-mapper-asl</include>
+                  <include>org.codehaus.jackson:jackson-core-asl</include>
+                </includes>
+              </artifactSet>
+              <relocations>
+                <relocation>
+                  <pattern>org.codehaus.jackson</pattern>
+                  <shadedPattern>parquet.org.codehaus.jackson</shadedPattern>
+                </relocation>
+              </relocations>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build> 
 </project>
diff --git a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftBytesOutputFormat.java b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftBytesOutputFormat.java
index aa2e23c..6b74514 100644
--- a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftBytesOutputFormat.java
+++ b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftBytesOutputFormat.java
@@ -22,19 +22,20 @@
 import org.apache.thrift.protocol.TProtocolFactory;
 
 import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.util.ContextUtil;
 
 public class ParquetThriftBytesOutputFormat extends ParquetOutputFormat<BytesWritable> {
 
   public static void setThriftClass(Job job, Class<? extends TBase<?, ?>> thriftClass) {
-    ThriftWriteSupport.setThriftClass(job.getConfiguration(), thriftClass);
+    ThriftWriteSupport.setThriftClass(ContextUtil.getConfiguration(job), thriftClass);
   }
 
   public static Class<? extends TBase<?,?>> getThriftClass(Job job) {
-    return ThriftWriteSupport.getThriftClass(job.getConfiguration());
+    return ThriftWriteSupport.getThriftClass(ContextUtil.getConfiguration(job));
   }
 
   public static <U extends TProtocol> void setTProtocolClass(Job job, Class<U> tProtocolClass) {
-    ThriftBytesWriteSupport.setTProtocolClass(job.getConfiguration(), tProtocolClass);
+    ThriftBytesWriteSupport.setTProtocolClass(ContextUtil.getConfiguration(job), tProtocolClass);
   }
 
   public ParquetThriftBytesOutputFormat() {
diff --git a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftInputFormat.java b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftInputFormat.java
index 64baca6..b042907 100644
--- a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftInputFormat.java
+++ b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftInputFormat.java
@@ -15,6 +15,9 @@
  */
 package parquet.hadoop.thrift;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+
 import parquet.hadoop.ParquetInputFormat;
 
 public class ParquetThriftInputFormat<T> extends ParquetInputFormat<T> {
@@ -23,4 +26,28 @@
   public ParquetThriftInputFormat() {
     super(ThriftReadSupport.class);
   }
+
+  /**
+   * Call this method when setting up your Hadoop job if reading into a Thrift object
+   * that is not encoded into the parquet-serialized thrift metadata (for example,
+   * writing with Apache Thrift, but reading back into Twitter Scrooge version of
+   * the same thrift definition, or a different but compatible Apache Thrift class).
+   * @param conf
+   * @param klass
+   */
+  public static <T> void setThriftClass(JobConf conf, Class<T> klass) {
+    conf.set(ThriftReadSupport.THRIFT_READ_CLASS_KEY, klass.getName());
+  }
+
+  /**
+   * Call this method when setting up your Hadoop job if reading into a Thrift object
+   * that is not encoded into the parquet-serialized thrift metadata (for example,
+   * writing with Apache Thrift, but reading back into Twitter Scrooge version of
+   * the same thrift definition, or a different but compatible Apache Thrift class).
+   * @param conf
+   * @param klass
+   */
+  public static  <T> void setThriftClass(Configuration conf, Class<T> klass) {
+    conf.set(ThriftReadSupport.THRIFT_READ_CLASS_KEY, klass.getName());
+  }
 }
diff --git a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftOutputFormat.java b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftOutputFormat.java
index 5b3cb4d..f9a96af 100644
--- a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftOutputFormat.java
+++ b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ParquetThriftOutputFormat.java
@@ -18,6 +18,7 @@
 import org.apache.thrift.TBase;
 
 import parquet.hadoop.ParquetOutputFormat;
+import parquet.hadoop.util.ContextUtil;
 
 /**
  *
@@ -28,11 +29,11 @@
 public class ParquetThriftOutputFormat<T extends TBase<?,?>> extends ParquetOutputFormat<T> {
 
   public static void setThriftClass(Job job, Class<? extends TBase<?,?>> thriftClass) {
-    ThriftWriteSupport.setThriftClass(job.getConfiguration(), thriftClass);
+    ThriftWriteSupport.setThriftClass(ContextUtil.getConfiguration(job), thriftClass);
   }
 
   public static Class<? extends TBase<?,?>> getThriftClass(Job job) {
-    return ThriftWriteSupport.getThriftClass(job.getConfiguration());
+    return ThriftWriteSupport.getThriftClass(ContextUtil.getConfiguration(job));
   }
 
   public ParquetThriftOutputFormat() {
diff --git a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftReadSupport.java b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftReadSupport.java
index 2567908..852fc3a 100644
--- a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftReadSupport.java
+++ b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftReadSupport.java
@@ -15,9 +15,13 @@
  */
 package parquet.hadoop.thrift;
 
+import java.lang.reflect.Constructor;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.thrift.TBase;
+import org.apache.thrift.protocol.TProtocol;
 
 import parquet.hadoop.api.ReadSupport;
 import parquet.io.api.RecordMaterializer;
@@ -25,15 +29,49 @@
 import parquet.thrift.TBaseRecordConverter;
 import parquet.thrift.ThriftMetaData;
 import parquet.thrift.ThriftRecordConverter;
+import parquet.thrift.struct.ThriftType.StructType;
 
 public class ThriftReadSupport<T> extends ReadSupport<T> {
+  private static final String RECORD_CONVERTER_DEFAULT = TBaseRecordConverter.class.getName();
+  public static final String THRIFT_READ_CLASS_KEY = "parquet.thrift.read.class";
+
+  /**
+   * A {@link ThriftRecordConverter} builds an object by working with {@link TProtocol}. The default
+   * implementation creates standard Apache Thrift {@link TBase} objects; to support alternatives, such
+   * as <a href="http://github.com/twitter/scrooge">Twiter's Scrooge</a>, a custom converter can be specified using this key
+   * (for example, ScroogeRecordConverter from parquet-scrooge).
+   */
+  private static final String RECORD_CONVERTER_CLASS_KEY = "parquet.thrift.converter.class";
+
+  /**
+   * A {@link ThriftRecordConverter} builds an object by working with {@link TProtocol}. The default
+   * implementation creates standard Apache Thrift {@link TBase} objects; to support alternatives, such
+   * as <a href="http://github.com/twitter/scrooge">Twiter's Scrooge</a>, a custom converter can be specified
+   * (for example, ScroogeRecordConverter from parquet-scrooge).
+   */
+  public static void setRecordConverterClass(JobConf conf,
+      Class<?> klass) {
+    conf.set(RECORD_CONVERTER_CLASS_KEY, klass.getName());
+  }
+
 
   @Override
   public parquet.hadoop.api.ReadSupport.ReadContext init(
       Configuration configuration, Map<String, String> keyValueMetaData,
-      MessageType fileSchema) {
-    // TODO: handle the requested schema
-    return new ReadContext(fileSchema);
+      MessageType fileMessageType) {
+    String partialSchemaString = configuration.get(ReadSupport.PARQUET_READ_SCHEMA);
+    MessageType requestedProjection = getSchemaForRead(fileMessageType, partialSchemaString);
+    return new ReadContext(requestedProjection);
+  }
+
+  @SuppressWarnings("unchecked")
+  private Class<T> getThriftClass(ThriftMetaData metadata, Configuration conf) throws ClassNotFoundException {
+    String className = conf.get(THRIFT_READ_CLASS_KEY, null);
+    if (className == null) {
+      return (Class<T>) metadata.getThriftClass();
+    } else {
+        return (Class<T>) Class.forName(className);
+    }
   }
 
   @Override
@@ -41,11 +79,19 @@
       Map<String, String> keyValueMetaData, MessageType fileSchema,
       parquet.hadoop.api.ReadSupport.ReadContext readContext) {
     final ThriftMetaData thriftMetaData = ThriftMetaData.fromExtraMetaData(keyValueMetaData);
-    final Class<T> thriftClass = (Class<T>)thriftMetaData.getThriftClass();
-    // TODO: handle the requested schema
-    ThriftRecordConverter<T> converter = new TBaseRecordConverter(thriftClass, fileSchema, thriftMetaData.getDescriptor());
-    return converter;
-  }
+    try {
+      final Class<T> thriftClass = getThriftClass(thriftMetaData, configuration);
 
+      String converterClassName = configuration.get(RECORD_CONVERTER_CLASS_KEY, RECORD_CONVERTER_DEFAULT);
+      @SuppressWarnings("unchecked")
+      Class<ThriftRecordConverter<T>> converterClass = (Class<ThriftRecordConverter<T>>) Class.forName(converterClassName);
+      Constructor<ThriftRecordConverter<T>> constructor =
+          converterClass.getConstructor(Class.class, MessageType.class, StructType.class);
+      ThriftRecordConverter<T> converter = constructor.newInstance(thriftClass, readContext.getRequestedSchema(), thriftMetaData.getDescriptor());
+      return converter;
+    } catch (Exception t) {
+      throw new RuntimeException("Unable to create Thrift Converter for Thrift metadata " + thriftMetaData, t);
+    }
+  }
 
 }
diff --git a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftToParquetFileWriter.java b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftToParquetFileWriter.java
index 79d215d..432c7f1 100644
--- a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftToParquetFileWriter.java
+++ b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftToParquetFileWriter.java
@@ -15,6 +15,7 @@
  */
 package parquet.hadoop.thrift;
 
+import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.hadoop.fs.Path;
@@ -30,7 +31,7 @@
  * @author Julien Le Dem
  *
  */
-public class ThriftToParquetFileWriter {
+public class ThriftToParquetFileWriter implements Closeable {
 
   private final RecordWriter<Void, BytesWritable> recordWriter;
   private final TaskAttemptContext taskAttemptContext;
@@ -52,7 +53,13 @@
     recordWriter.write(null, bytes);
   }
 
-  public void close() throws IOException, InterruptedException {
-    recordWriter.close(taskAttemptContext);
+  @Override
+  public void close() throws IOException {
+    try {
+      recordWriter.close(taskAttemptContext);
+    } catch (InterruptedException e) {
+      Thread.interrupted();
+      throw new IOException("The thread was interrupted", e);
+    }
   }
 }
diff --git a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftWriteSupport.java b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftWriteSupport.java
index 8d83693..b14959f 100644
--- a/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftWriteSupport.java
+++ b/parquet-thrift/src/main/java/parquet/hadoop/thrift/ThriftWriteSupport.java
@@ -68,7 +68,7 @@
     ThriftSchemaConverter thriftSchemaConverter = new ThriftSchemaConverter();
     this.thriftStruct = thriftSchemaConverter.toStructType(thriftClass);
     this.schema = thriftSchemaConverter.convert(thriftClass);
-    final Map<String, String> extraMetaData = new ThriftMetaData(thriftClass, thriftStruct).toExtraMetaData();
+    final Map<String, String> extraMetaData = new ThriftMetaData(thriftClass.getName(), thriftStruct).toExtraMetaData();
     // adding the Pig schema as it would have been mapped from thrift
     new PigMetaData(new ThriftToPig(thriftClass).toSchema()).addToMetaData(extraMetaData);
     return new WriteContext(schema, extraMetaData);
diff --git a/parquet-thrift/src/main/java/parquet/thrift/ParquetProtocol.java b/parquet-thrift/src/main/java/parquet/thrift/ParquetProtocol.java
index 4d6f46f..f77810a 100644
--- a/parquet-thrift/src/main/java/parquet/thrift/ParquetProtocol.java
+++ b/parquet-thrift/src/main/java/parquet/thrift/ParquetProtocol.java
@@ -15,6 +15,7 @@
  */
 package parquet.thrift;
 
+import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 
 import org.apache.thrift.TException;
@@ -38,7 +39,16 @@
 
   ParquetProtocol() {
     super(null);
-    this.message = getClass().getName();
+    this.message = "in " + getClassInfo();
+  }
+
+  private String getClassInfo() {
+    final Class<? extends ParquetProtocol> clazz = getClass();
+    final Method enclosingMethod = clazz.getEnclosingMethod();
+    if (enclosingMethod != null) {
+      return clazz.getName() + " in " + enclosingMethod.toGenericString();
+    }
+    return clazz.getName();
   }
 
   /**
@@ -46,7 +56,11 @@
    */
   ParquetProtocol(String name) {
     super(null);
-    this.message = name + " " + getClass().getName();
+    this.message = "when we expected " + name + " in " + getClassInfo();
+  }
+
+  private UnsupportedOperationException exception() {
+    return new UnsupportedOperationException(new Exception().getStackTrace()[1].getMethodName() + " was called " + message);
   }
 
   /** WRITE */
@@ -58,204 +72,204 @@
 
   @Override
   public void writeMessageEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeStructBegin(TStruct struct) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeStructEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeFieldBegin(TField field) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeFieldEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeFieldStop() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeMapBegin(TMap map) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeMapEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeListBegin(TList list) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeListEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeSetBegin(TSet set) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeSetEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeBool(boolean b) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeByte(byte b) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeI16(short i16) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeI32(int i32) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeI64(long i64) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeDouble(double dub) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeString(String str) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void writeBinary(ByteBuffer buf) throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   /** READ */
 
   @Override
   public TMessage readMessageBegin() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void readMessageEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public TStruct readStructBegin() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void readStructEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public TField readFieldBegin() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void readFieldEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public TMap readMapBegin() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void readMapEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public TList readListBegin() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void readListEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public TSet readSetBegin() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public void readSetEnd() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public boolean readBool() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public byte readByte() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public short readI16() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public int readI32() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public long readI64() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public double readDouble() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public String readString() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
   @Override
   public ByteBuffer readBinary() throws TException {
-    throw new UnsupportedOperationException(message);
+    throw exception();
   }
 
 }
diff --git a/parquet-thrift/src/main/java/parquet/thrift/ProtocolReadToWrite.java b/parquet-thrift/src/main/java/parquet/thrift/ProtocolReadToWrite.java
index 320f61e..24fce60 100644
--- a/parquet-thrift/src/main/java/parquet/thrift/ProtocolReadToWrite.java
+++ b/parquet-thrift/src/main/java/parquet/thrift/ProtocolReadToWrite.java
@@ -79,7 +79,7 @@
       out.writeI64(in.readI64());
       break;
     case TType.STRING:
-      out.writeString(in.readString());
+      out.writeBinary(in.readBinary());
       break;
     case TType.VOID:
       break;
diff --git a/parquet-thrift/src/main/java/parquet/thrift/TBaseRecordConverter.java b/parquet-thrift/src/main/java/parquet/thrift/TBaseRecordConverter.java
index bf58921..fa677b8 100644
--- a/parquet-thrift/src/main/java/parquet/thrift/TBaseRecordConverter.java
+++ b/parquet-thrift/src/main/java/parquet/thrift/TBaseRecordConverter.java
@@ -19,6 +19,7 @@
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TProtocol;
 
+import parquet.io.ParquetDecodingException;
 import parquet.schema.MessageType;
 import parquet.thrift.struct.ThriftType.StructType;
 
@@ -28,15 +29,15 @@
     super(new ThriftReader<T>() {
       @Override
       public T readOneRecord(TProtocol protocol) throws TException {
-        try {
-          T thriftObject = thriftClass.newInstance();
-          thriftObject.read(protocol);
-          return thriftObject;
-        } catch (InstantiationException e) {
-          throw new RuntimeException(e);
-        } catch (IllegalAccessException e) {
-          throw new RuntimeException(e);
-        }
+          try {
+            T thriftObject = thriftClass.newInstance();
+            thriftObject.read(protocol);
+            return thriftObject;
+          } catch (InstantiationException e) {
+            throw new ParquetDecodingException("Could not instantiate Thrift " + thriftClass, e);
+          } catch (IllegalAccessException e) {
+            throw new ParquetDecodingException("Thrift class or constructor not public " + thriftClass, e);
+          }
       }
     }, thriftClass.getSimpleName(), parquetSchema, thriftType);
   }
diff --git a/parquet-thrift/src/main/java/parquet/thrift/ThriftMetaData.java b/parquet-thrift/src/main/java/parquet/thrift/ThriftMetaData.java
index bfd7f06..5eb0915 100644
--- a/parquet-thrift/src/main/java/parquet/thrift/ThriftMetaData.java
+++ b/parquet-thrift/src/main/java/parquet/thrift/ThriftMetaData.java
@@ -19,24 +19,43 @@
 
 import org.apache.thrift.TBase;
 
+import parquet.Log;
 import parquet.hadoop.BadConfigurationException;
 import parquet.thrift.struct.ThriftType;
 import parquet.thrift.struct.ThriftType.StructType;
 
 
 public class ThriftMetaData {
+  private static final Log LOG = Log.getLog(ThriftMetaData.class);
 
   private static final String THRIFT_CLASS = "thrift.class";
   private static final String THRIFT_DESCRIPTOR = "thrift.descriptor";
-  private final Class<?> thriftClass;
+  private Class<?> thriftClass;
+  private final String thriftClassName;
   private final StructType descriptor;
 
-  public ThriftMetaData(Class<?> thriftClass, StructType descriptor) {
-    this.thriftClass = thriftClass;
+  public ThriftMetaData(String thriftClassName, StructType descriptor) {
+    this.thriftClassName = thriftClassName;
     this.descriptor = descriptor;
   }
 
+  /**
+   * Get the Thrift Class encoded in the metadata.
+   * @return Thrift Class encoded in the metadata.
+   * @throws BadConfigurationException if the encoded class does not
+   * extend TBase or is not available in the current classloader.
+   */
   public Class<?> getThriftClass() {
+    if (thriftClass == null) {
+      try {
+        thriftClass = Class.forName(thriftClassName);
+        if (!TBase.class.isAssignableFrom(thriftClass)) {
+          throw new BadConfigurationException("Provided class " + thriftClassName + " does not extend TBase");
+        }
+      } catch (ClassNotFoundException e) {
+        throw new BadConfigurationException("Could not instantiate thrift class " + thriftClassName, e);
+      }
+    }
     return thriftClass;
   }
 
@@ -44,6 +63,13 @@
     return descriptor;
   }
 
+  /**
+   * Reads ThriftMetadata from the parquet file footer.
+   *
+   *
+   * @param extraMetaData  extraMetaData field of the parquet footer
+   * @return
+   */
   public static ThriftMetaData fromExtraMetaData(
       Map<String, String> extraMetaData) {
     final String thriftClassName = extraMetaData.get(THRIFT_CLASS);
@@ -51,27 +77,18 @@
     if (thriftClassName == null && thriftDescriptorString == null) {
       return null;
     }
-    Class<?> thriftClass;
-    try {
-      thriftClass = Class.forName(thriftClassName);
-      if (!TBase.class.isAssignableFrom(thriftClass)) {
-        throw new BadConfigurationException("Provided class " + thriftClassName + " does not extend TBase");
-      }
-    } catch (ClassNotFoundException e) {
-      throw new BadConfigurationException("Could not instanciate thrift class " + thriftClassName, e);
-    }
     final StructType descriptor;
     try {
       descriptor = (StructType)ThriftType.fromJSON(thriftDescriptorString);
     } catch (RuntimeException e) {
       throw new BadConfigurationException("Could not read the thrift descriptor " + thriftDescriptorString, e);
     }
-    return new ThriftMetaData(thriftClass, descriptor);
+    return new ThriftMetaData(thriftClassName, descriptor);
   }
 
   public Map<String, String> toExtraMetaData() {
     final Map<String, String> map = new HashMap<String, String>();
-    map.put(THRIFT_CLASS, thriftClass.getName());
+    map.put(THRIFT_CLASS, getThriftClass().getName());
     map.put(THRIFT_DESCRIPTOR, descriptor.toJSON());
     return map;
   }
diff --git a/parquet-thrift/src/main/java/parquet/thrift/ThriftRecordConverter.java b/parquet-thrift/src/main/java/parquet/thrift/ThriftRecordConverter.java
index 9cc7bf1..6043ad9 100644
--- a/parquet-thrift/src/main/java/parquet/thrift/ThriftRecordConverter.java
+++ b/parquet-thrift/src/main/java/parquet/thrift/ThriftRecordConverter.java
@@ -708,17 +708,26 @@
       this.tStruct = new TStruct(name);
       this.thriftType = (StructType)field.getType();
       this.schemaSize = parquetSchema.getFieldCount();
-      if (schemaSize != thriftType.getChildren().size()) {
-        throw new IllegalArgumentException("schema sizes don't match:\n" + parquetSchema + "\n" + thriftType);
-      }
       this.converters = new Converter[this.schemaSize];
+      List<ThriftField> thriftChildren = thriftType.getChildren();
       for (int i = 0; i < schemaSize; i++) {
-        ThriftField childField = thriftType.getChildren().get(i);
-        Type type = parquetSchema.getType(i);
-        if (type.isPrimitive()) {
-          converters[i] = new PrimitiveFieldHandler(newConverter(events, type, childField).asPrimitiveConverter(), childField, events);
+        Type schemaType = parquetSchema.getType(i);
+        String fieldName = schemaType.getName();
+        ThriftField matchingThrift = null;
+        for (ThriftField childField: thriftChildren) {
+          String thriftChildName = childField.getName();
+          if (thriftChildName != null && thriftChildName.equalsIgnoreCase(fieldName)) {
+            matchingThrift = childField;
+            break;
+          }
+        }
+        if (matchingThrift == null) {
+        	throw new IllegalArgumentException("schema mismatch :: cannot find Thrift field for column [" + fieldName + "]");
+        }
+        if (schemaType.isPrimitive()) {
+        	converters[i] = new PrimitiveFieldHandler(newConverter(events, schemaType, matchingThrift).asPrimitiveConverter(), matchingThrift, events);
         } else {
-          converters[i] = new GroupFieldhandler(newConverter(events, type, childField).asGroupConverter(), childField, events);
+        	converters[i] = new GroupFieldhandler(newConverter(events, schemaType, matchingThrift).asGroupConverter(), matchingThrift, events);
         }
       }
     }
diff --git a/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestInputOutputFormat.java b/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestInputOutputFormat.java
index b003097..074dff6 100644
--- a/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestInputOutputFormat.java
+++ b/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestInputOutputFormat.java
@@ -63,7 +63,7 @@
 
   public static class MyMapper extends Mapper<LongWritable, Text, Void, AddressBook> {
 
-    public void run(org.apache.hadoop.mapreduce.Mapper<LongWritable,Text,Void,AddressBook>.Context context) throws IOException ,InterruptedException {
+    public void run(org.apache.hadoop.mapreduce.Mapper<LongWritable,Text,Void,AddressBook>.Context context) throws IOException, InterruptedException {
       for (int i = 0; i < 10; i++) {
         AddressBook a = TestInputOutputFormat.nextAddressbook(i);
         context.write(null, a);
diff --git a/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestParquetToThriftReadProjection.java b/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestParquetToThriftReadProjection.java
new file mode 100644
index 0000000..b78c640
--- /dev/null
+++ b/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestParquetToThriftReadProjection.java
@@ -0,0 +1,123 @@
+/**
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.thrift;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TIOStreamTransport;
+import org.junit.Test;
+
+import parquet.Log;
+import parquet.hadoop.api.ReadSupport;
+import parquet.hadoop.util.ContextUtil;
+
+import com.twitter.data.proto.tutorial.thrift.AddressBook;
+import com.twitter.data.proto.tutorial.thrift.Name;
+import com.twitter.data.proto.tutorial.thrift.Person;
+import com.twitter.data.proto.tutorial.thrift.PhoneNumber;
+
+public class TestParquetToThriftReadProjection {
+	
+  private static final Log LOG = Log.getLog(TestParquetToThriftReadProjection.class);
+
+   
+  @Test
+  public void testThriftOptionalFieldsWithReadProjection() throws IOException, InterruptedException, TException {
+  	
+    final Path parquetFile = new Path("target/test/TestParquetToThriftReadProjection/file.parquet");
+    final Configuration conf = new Configuration();
+    final FileSystem fs = parquetFile.getFileSystem(conf);
+    if (fs.exists(parquetFile)) {
+      fs.delete(parquetFile, true);
+    }
+    
+    //create a test file
+    final TProtocolFactory protocolFactory = new TCompactProtocol.Factory();
+    final TaskAttemptID taskId = new TaskAttemptID("local", 0, true, 0, 0);
+    final ThriftToParquetFileWriter w = new ThriftToParquetFileWriter(parquetFile, new TaskAttemptContext(conf, taskId), protocolFactory, AddressBook.class);
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    final TProtocol protocol = protocolFactory.getProtocol(new TIOStreamTransport(baos));
+    final AddressBook a = new AddressBook(
+        Arrays.asList(
+            new Person(
+                new Name("Bob", "Roberts"),
+                0,
+                "bob.roberts@example.com",
+                Arrays.asList(new PhoneNumber("1234567890")))));
+    a.write(protocol);
+    w.write(new BytesWritable(baos.toByteArray()));
+    w.close();
+
+    // test with projection
+    final String readProjectionSchema = "message AddressBook {\n" + 
+    		"  optional group persons {\n" + 
+    		"    repeated group persons_tuple {\n" + 
+    		"      required group name {\n" + 
+    		"        optional binary first_name;\n" + 
+    		"        optional binary last_name;\n" + 
+    		"      }\n" + 
+    		"      optional int32 id;\n" + 
+    		"    }\n" + 
+    		"  }\n" + 
+    		"}";
+    conf.set(ReadSupport.PARQUET_READ_SCHEMA, readProjectionSchema);
+    final ParquetThriftInputFormat<AddressBook> parquetThriftInputFormat = new ParquetThriftInputFormat<AddressBook>();
+    final Job job = new Job(conf, "read");
+    job.setInputFormatClass(ParquetThriftInputFormat.class);
+    ParquetThriftInputFormat.setInputPaths(job, parquetFile);
+    final JobID jobID = new JobID("local", 1);
+    List<InputSplit> splits = parquetThriftInputFormat.getSplits(new JobContext(ContextUtil.getConfiguration(job), jobID));
+    AddressBook expected = a.deepCopy();
+    for (Person person: expected.getPersons()) {
+    	person.unsetEmail();
+    	person.unsetPhones();
+    }
+    AddressBook readValue = null;
+    for (InputSplit split : splits) {
+      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(ContextUtil.getConfiguration(job), new TaskAttemptID(new TaskID(jobID, true, 1), 0));
+      final RecordReader<Void, AddressBook> reader = parquetThriftInputFormat.createRecordReader(split, taskAttemptContext);
+      reader.initialize(split, taskAttemptContext);
+      if (reader.nextKeyValue()) {
+        readValue = reader.getCurrentValue();
+        LOG.info(readValue);
+      } 
+    }
+    assertEquals(expected, readValue);
+
+  }
+
+}
diff --git a/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestThriftToParquetFileWriter.java b/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestThriftToParquetFileWriter.java
index 27e4627..15c6290 100644
--- a/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestThriftToParquetFileWriter.java
+++ b/parquet-thrift/src/test/java/parquet/hadoop/thrift/TestThriftToParquetFileWriter.java
@@ -49,6 +49,7 @@
 import com.twitter.data.proto.tutorial.thrift.Name;
 import com.twitter.data.proto.tutorial.thrift.Person;
 import com.twitter.data.proto.tutorial.thrift.PhoneNumber;
+import parquet.hadoop.util.ContextUtil;
 
 public class TestThriftToParquetFileWriter {
   private static final Log LOG = Log
@@ -87,11 +88,11 @@
     Job job = new Job();
     ExampleInputFormat.addInputPath(job, fileToCreate);
     final JobID jobID = new JobID("local", 1);
-    List<InputSplit> splits = exampleInputFormat.getSplits(new JobContext(job.getConfiguration(), jobID));
+    List<InputSplit> splits = exampleInputFormat.getSplits(new JobContext(ContextUtil.getConfiguration(job), jobID));
     int i = 0;
     for (InputSplit split : splits) {
       LOG.info(split);
-      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID(new TaskID(jobID, true, i), 0));
+      TaskAttemptContext taskAttemptContext = new TaskAttemptContext(ContextUtil.getConfiguration(job), new TaskAttemptID(new TaskID(jobID, true, i), 0));
       final RecordReader<Void, Group> reader = exampleInputFormat.createRecordReader(split, taskAttemptContext);
       reader.initialize(split, taskAttemptContext);
       while (reader.nextKeyValue()) {
diff --git a/parquet-thrift/src/test/java/parquet/thrift/TestParquetReadProtocol.java b/parquet-thrift/src/test/java/parquet/thrift/TestParquetReadProtocol.java
index bb24c0c..bf06799 100644
--- a/parquet-thrift/src/test/java/parquet/thrift/TestParquetReadProtocol.java
+++ b/parquet-thrift/src/test/java/parquet/thrift/TestParquetReadProtocol.java
@@ -141,8 +141,8 @@
     final ThriftSchemaConverter schemaConverter = new ThriftSchemaConverter();
     final MessageType schema = schemaConverter.convert(thriftClass);
     LOG.info(schema);
-    final MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema);
-    final ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 10000);
+    final MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
+    final ColumnWriteStoreImpl columns = new ColumnWriteStoreImpl(memPageStore, 10000, 10000, false);
     final RecordConsumer recordWriter = columnIO.getRecordWriter(columns);
     final StructType thriftType = schemaConverter.toStructType(thriftClass);
     ParquetWriteProtocol parquetWriteProtocol = new ParquetWriteProtocol(recordWriter, columnIO, thriftType);
diff --git a/parquet-thrift/src/test/java/parquet/thrift/TestThriftSchemaConverter.java b/parquet-thrift/src/test/java/parquet/thrift/TestThriftSchemaConverter.java
index bb8e84d..f2347be 100644
--- a/parquet-thrift/src/test/java/parquet/thrift/TestThriftSchemaConverter.java
+++ b/parquet-thrift/src/test/java/parquet/thrift/TestThriftSchemaConverter.java
@@ -64,4 +64,5 @@
     final ThriftType fromJSON = StructType.fromJSON(json);
     assertEquals(json, fromJSON.toJSON());
   }
+
 }
diff --git a/pom.xml b/pom.xml
index ae93fba..9c83047 100644
--- a/pom.xml
+++ b/pom.xml
@@ -59,28 +59,38 @@
     <maven.compiler.source>1.6</maven.compiler.source>
     <maven.compiler.target>1.6</maven.compiler.target>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <maven-jar-plugin.version>2.4</maven-jar-plugin.version>
+    <jackson.version>1.9.11</jackson.version>
+    <hadoop.version>1.1.0</hadoop.version>
   </properties>
 
   <modules>
+    <module>parquet-avro</module>
+    <module>parquet-cascading</module>
     <module>parquet-column</module>
+    <module>parquet-common</module>
+    <module>parquet-encoding</module>
+    <module>parquet-generator</module>
     <module>parquet-hadoop</module>
     <module>parquet-pig</module>
+    <module>parquet-scrooge</module>
     <module>parquet-thrift</module>
+    <module>parquet-test-hadoop2</module>
   </modules>
 
   <dependencies>
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-      <version>1.9.11</version>
-    </dependency>
-
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <version>4.10</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>commons-httpclient</groupId>
+      <artifactId>commons-httpclient</artifactId>
+      <version>3.0.1</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
@@ -160,6 +170,19 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>buildnumber-maven-plugin</artifactId>
+        <version>1.1</version>
+        <executions>
+          <execution>
+            <phase>validate</phase>
+            <goals>
+              <goal>create</goal>
+            </goals>
+         </execution>
+       </executions>
+      </plugin>
     </plugins>
   </build>