You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ju...@apache.org on 2015/11/04 18:57:33 UTC

[1/4] parquet-mr git commit: PARQUET-77: ByteBuffer use in read and write paths

Repository: parquet-mr
Updated Branches:
  refs/heads/master 5a45ae3b1 -> 6b605a4ea


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java
index 0a0b316..2eab54a 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageWriteStore.java
@@ -42,6 +42,7 @@ import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.CodecFactory.BytesCompressor;
 import org.apache.parquet.io.ParquetEncodingException;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.bytes.ByteBufferAllocator;
 
 class ColumnChunkPageWriteStore implements PageWriteStore {
   private static final Log LOG = Log.getLog(ColumnChunkPageWriteStore.class);
@@ -65,10 +66,14 @@ class ColumnChunkPageWriteStore implements PageWriteStore {
     private Set<Encoding> encodings = new HashSet<Encoding>();
 
     private Statistics totalStatistics;
+    private final ByteBufferAllocator allocator;
 
-    private ColumnChunkPageWriter(ColumnDescriptor path, BytesCompressor compressor, int pageSize) {
+    private ColumnChunkPageWriter(ColumnDescriptor path,
+                                  BytesCompressor compressor,
+                                  ByteBufferAllocator allocator) {
       this.path = path;
       this.compressor = compressor;
+      this.allocator = allocator;
       this.buf = new ConcatenatingByteArrayCollector();
       this.totalStatistics = getStatsBasedOnType(this.path.getType());
     }
@@ -84,14 +89,14 @@ class ColumnChunkPageWriteStore implements PageWriteStore {
       if (uncompressedSize > Integer.MAX_VALUE) {
         throw new ParquetEncodingException(
             "Cannot write page larger than Integer.MAX_VALUE bytes: " +
-            uncompressedSize);
+                uncompressedSize);
       }
       BytesInput compressedBytes = compressor.compress(bytes);
       long compressedSize = compressedBytes.size();
       if (compressedSize > Integer.MAX_VALUE) {
         throw new ParquetEncodingException(
             "Cannot write compressed page larger than Integer.MAX_VALUE bytes: "
-            + compressedSize);
+                + compressedSize);
       }
       tempOutputStream.reset();
       parquetMetadataConverter.writeDataPageHeader(
@@ -151,10 +156,10 @@ class ColumnChunkPageWriteStore implements PageWriteStore {
       // we only allocate one buffer to copy into instead of multiple.
       buf.collect(
           BytesInput.concat(
-            BytesInput.from(tempOutputStream),
-            repetitionLevels,
-            definitionLevels,
-            compressedData)
+              BytesInput.from(tempOutputStream),
+              repetitionLevels,
+              definitionLevels,
+              compressedData)
       );
       encodings.add(dataEncoding);
     }
@@ -163,7 +168,7 @@ class ColumnChunkPageWriteStore implements PageWriteStore {
       if (size > Integer.MAX_VALUE) {
         throw new ParquetEncodingException(
             "Cannot write page larger than " + Integer.MAX_VALUE + " bytes: " +
-            size);
+                size);
       }
       return (int)size;
     }
@@ -186,10 +191,10 @@ class ColumnChunkPageWriteStore implements PageWriteStore {
             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())
-                    : ""));
+                + (dictionaryPage != null ? String.format(
+                ", dic { %,d entries, %,dB raw, %,dB comp}",
+                dictionaryPage.getDictionarySize(), dictionaryPage.getUncompressedSize(), dictionaryPage.getDictionarySize())
+                : ""));
       }
       encodings.clear();
       pageCount = 0;
@@ -215,15 +220,16 @@ class ColumnChunkPageWriteStore implements PageWriteStore {
     public String memUsageString(String prefix) {
       return buf.memUsageString(prefix + " ColumnChunkPageWriter");
     }
+
   }
 
   private final Map<ColumnDescriptor, ColumnChunkPageWriter> writers = new HashMap<ColumnDescriptor, ColumnChunkPageWriter>();
   private final MessageType schema;
 
-  public ColumnChunkPageWriteStore(BytesCompressor compressor, MessageType schema, int pageSize) {
+  public ColumnChunkPageWriteStore(BytesCompressor compressor, MessageType schema, ByteBufferAllocator allocator) {
     this.schema = schema;
     for (ColumnDescriptor path : schema.getColumns()) {
-      writers.put(path,  new ColumnChunkPageWriter(path, compressor, pageSize));
+      writers.put(path,  new ColumnChunkPageWriter(path, compressor, allocator));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java
new file mode 100644
index 0000000..bb711da
--- /dev/null
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/DirectCodecFactory.java
@@ -0,0 +1,522 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.parquet.hadoop;
+
+
+
+import java.lang.reflect.Method;
+import java.lang.reflect.InvocationTargetException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.pool.BasePoolableObjectFactory;
+import org.apache.commons.pool.impl.GenericObjectPool;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.xerial.snappy.Snappy;
+
+import org.apache.parquet.bytes.ByteBufferAllocator;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.Log;
+import org.apache.parquet.ParquetRuntimeException;
+import org.apache.parquet.Preconditions;
+
+/**
+ * Factory to produce compressors and decompressors that operate on java
+ * direct memory, without requiring a copy into heap memory (where possible).
+ */
+class DirectCodecFactory extends CodecFactory implements AutoCloseable {
+  private static final Log LOG = Log.getLog(DirectCodecFactory.class);
+
+  private final ByteBufferAllocator allocator;
+
+  // Any of these can be null depending on the version of hadoop on the classpath
+  private static final Class<?> DIRECT_DECOMPRESSION_CODEC_CLASS;
+  private static final Method DECOMPRESS_METHOD;
+  private static final Method CREATE_DIRECT_DECOMPRESSOR_METHOD;
+
+  static {
+    Class<?> tempClass = null;
+    Method tempCreateMethod = null;
+    Method tempDecompressMethod = null;
+    try {
+      tempClass = Class.forName("org.apache.hadoop.io.compress.DirectDecompressionCodec");
+      tempCreateMethod = tempClass.getMethod("createDirectDecompressor");
+      tempDecompressMethod = tempClass.getMethod("decompress", ByteBuffer.class, ByteBuffer.class);
+    } catch (ClassNotFoundException e) {
+      // do nothing, the class will just be assigned null
+    } catch (NoSuchMethodException e) {
+      // do nothing, the method will just be assigned null
+    }
+    DIRECT_DECOMPRESSION_CODEC_CLASS = tempClass;
+    CREATE_DIRECT_DECOMPRESSOR_METHOD = tempCreateMethod;
+    DECOMPRESS_METHOD = tempDecompressMethod;
+  }
+
+  /**
+   * See docs on CodecFactory#createDirectCodecFactory which is how this class is
+   * exposed publicly and is just a pass-through factory method for this constructor
+   * to hide the rest of this class from public access.
+   */
+  DirectCodecFactory(Configuration config, ByteBufferAllocator allocator, int pageSize) {
+    super(config, pageSize);
+    Preconditions.checkNotNull(allocator, "allocator");
+    Preconditions.checkState(allocator.isDirect(),
+        "A %s requires a direct buffer allocator be provided.",
+        getClass().getSimpleName());
+    this.allocator = allocator;
+  }
+
+  private ByteBuffer ensure(ByteBuffer buffer, int size) {
+    if (buffer == null) {
+      buffer = allocator.allocate(size);
+    } else if (buffer.capacity() >= size) {
+      buffer.clear();
+    } else {
+      release(buffer);
+      buffer = allocator.allocate(size);
+    }
+    return buffer;
+  }
+
+  ByteBuffer release(ByteBuffer buffer) {
+    if (buffer != null) {
+      allocator.release(buffer);
+    }
+    return null;
+  }
+
+  @Override
+  protected BytesCompressor createCompressor(final CompressionCodecName codecName) {
+
+    CompressionCodec codec = getCodec(codecName);
+    if (codec == null) {
+      return new NoopCompressor();
+    } else if (codecName == CompressionCodecName.SNAPPY) {
+      // avoid using the default Snappy codec since it allocates direct buffers at awkward spots.
+      return new SnappyCompressor();
+    } else {
+      // todo: create class similar to the SnappyCompressor for zlib and exclude it as
+      // snappy is above since it also generates allocateDirect calls.
+      return new HeapBytesCompressor(codecName);
+    }
+  }
+
+  @Override
+  protected BytesDecompressor createDecompressor(final CompressionCodecName codecName) {
+    CompressionCodec codec = getCodec(codecName);
+    if (codec == null) {
+      return new NoopDecompressor();
+    } else if (codecName == CompressionCodecName.SNAPPY ) {
+      return new SnappyDecompressor();
+    } else if (DirectCodecPool.INSTANCE.codec(codec).supportsDirectDecompression()) {
+      return new FullDirectDecompressor(codecName);
+    } else {
+      return new IndirectDecompressor(codec);
+    }
+  }
+
+  public void close() {
+    release();
+  }
+
+  /**
+   * Wrapper around legacy hadoop compressors that do not implement a direct memory
+   * based version of the decompression algorithm.
+   */
+  public class IndirectDecompressor extends BytesDecompressor {
+    private final Decompressor decompressor;
+
+    public IndirectDecompressor(CompressionCodec codec) {
+      this.decompressor = DirectCodecPool.INSTANCE.codec(codec).borrowDecompressor();
+    }
+
+    @Override
+    public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException {
+      decompressor.reset();
+      byte[] inputBytes = bytes.toByteArray();
+      decompressor.setInput(inputBytes, 0, inputBytes.length);
+      byte[] output = new byte[uncompressedSize];
+      decompressor.decompress(output, 0, uncompressedSize);
+      return BytesInput.from(output);
+    }
+
+    @Override
+    public void decompress(ByteBuffer input, int compressedSize, ByteBuffer output, int uncompressedSize)
+        throws IOException {
+
+      decompressor.reset();
+      byte[] inputBytes = new byte[compressedSize];
+      input.position(0);
+      input.get(inputBytes);
+      decompressor.setInput(inputBytes, 0, inputBytes.length);
+      byte[] outputBytes = new byte[uncompressedSize];
+      decompressor.decompress(outputBytes, 0, uncompressedSize);
+      output.clear();
+      output.put(outputBytes);
+    }
+
+    @Override
+    protected void release() {
+      DirectCodecPool.INSTANCE.returnDecompressor(decompressor);
+    }
+  }
+
+  /**
+   * Wrapper around new Hadoop compressors that implement a direct memory
+   * based version of a particular decompression algorithm. To maintain
+   * compatibility with Hadoop 1.x these classes that implement
+   * {@link org.apache.hadoop.io.compress.DirectDecompressionCodec}
+   * are currently retrieved and have their decompression method invoked
+   * with reflection.
+   */
+  public class FullDirectDecompressor extends BytesDecompressor {
+    private final Object decompressor;
+    private HeapBytesDecompressor extraDecompressor;
+    public FullDirectDecompressor(CompressionCodecName codecName){
+      CompressionCodec codec = getCodec(codecName);
+      this.decompressor = DirectCodecPool.INSTANCE.codec(codec).borrowDirectDecompressor();
+      this.extraDecompressor = new HeapBytesDecompressor(codecName);
+    }
+
+    @Override
+    public BytesInput decompress(BytesInput compressedBytes, int uncompressedSize) throws IOException {
+    	return extraDecompressor.decompress(compressedBytes, uncompressedSize);
+    }
+
+    @Override
+    public void decompress(ByteBuffer input, int compressedSize, ByteBuffer output, int uncompressedSize)
+        throws IOException {
+      output.clear();
+      try {
+        DECOMPRESS_METHOD.invoke(decompressor, (ByteBuffer) input.limit(compressedSize), (ByteBuffer) output.limit(uncompressedSize));
+      } catch (IllegalAccessException e) {
+        throw new DirectCodecPool.ParquetCompressionCodecException(e);
+      } catch (InvocationTargetException e) {
+        throw new DirectCodecPool.ParquetCompressionCodecException(e);
+      }
+      output.position(uncompressedSize);
+    }
+
+    @Override
+    protected void release() {
+      DirectCodecPool.INSTANCE.returnDirectDecompressor(decompressor);
+      extraDecompressor.release();
+    }
+
+  }
+
+  public class NoopDecompressor extends BytesDecompressor {
+
+    @Override
+    public void decompress(ByteBuffer input, int compressedSize, ByteBuffer output, int uncompressedSize)
+        throws IOException {
+      Preconditions.checkArgument(compressedSize == uncompressedSize,
+          "Non-compressed data did not have matching compressed and uncompressed sizes.");
+      output.clear();
+      output.put((ByteBuffer) input.duplicate().position(0).limit(compressedSize));
+    }
+
+    @Override
+    public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException {
+      return bytes;
+    }
+
+    @Override
+    protected void release() {}
+
+  }
+
+  public class SnappyDecompressor extends BytesDecompressor {
+
+    private HeapBytesDecompressor extraDecompressor;
+    public SnappyDecompressor() {
+      this.extraDecompressor = new HeapBytesDecompressor(CompressionCodecName.SNAPPY);
+    }
+
+    @Override
+    public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException {
+      return extraDecompressor.decompress(bytes, uncompressedSize);
+    }
+
+    @Override
+    public void decompress(ByteBuffer src, int compressedSize, ByteBuffer dst, int uncompressedSize) throws IOException {
+      dst.clear();
+      int size = Snappy.uncompress(src, dst);
+      dst.limit(size);
+    }
+
+    @Override
+    protected void release() {}
+  }
+
+  public class SnappyCompressor extends BytesCompressor {
+
+    // TODO - this outgoing buffer might be better off not being shared, this seems to
+    // only work because of an extra copy currently happening where this interface is
+    // be consumed
+    private ByteBuffer incoming;
+    private ByteBuffer outgoing;
+
+    /**
+     * Compress a given buffer of bytes
+     * @param bytes
+     * @return
+     * @throws IOException
+     */
+    @Override
+    public BytesInput compress(BytesInput bytes) throws IOException {
+      int maxOutputSize = Snappy.maxCompressedLength((int) bytes.size());
+      ByteBuffer bufferIn = bytes.toByteBuffer();
+      outgoing = ensure(outgoing, maxOutputSize);
+      final int size;
+      if (bufferIn.isDirect()) {
+        size = Snappy.compress(bufferIn, outgoing);
+      } else {
+        // Snappy library requires buffers be direct
+        this.incoming = ensure(this.incoming, (int) bytes.size());
+        this.incoming.put(bufferIn);
+        this.incoming.flip();
+        size = Snappy.compress(this.incoming, outgoing);
+      }
+
+      return BytesInput.from(outgoing, 0, (int) size);
+    }
+
+    @Override
+    public CompressionCodecName getCodecName() {
+      return CompressionCodecName.SNAPPY;
+    }
+
+    @Override
+    protected void release() {
+      outgoing = DirectCodecFactory.this.release(outgoing);
+      incoming = DirectCodecFactory.this.release(incoming);
+    }
+
+  }
+
+  public static class NoopCompressor extends BytesCompressor {
+
+    public NoopCompressor() {}
+
+    @Override
+    public BytesInput compress(BytesInput bytes) throws IOException {
+      return bytes;
+    }
+
+    @Override
+    public CompressionCodecName getCodecName() {
+      return CompressionCodecName.UNCOMPRESSED;
+    }
+
+    @Override
+    protected void release() {}
+  }
+
+  static class DirectCodecPool {
+
+    public static final DirectCodecPool INSTANCE = new DirectCodecPool();
+
+    private final Map<CompressionCodec, CodecPool> codecs =
+        Collections.synchronizedMap(new HashMap<CompressionCodec, CodecPool>());
+    private final Map<Class<?>, GenericObjectPool> directDePools = Collections
+        .synchronizedMap(new HashMap<Class<?>, GenericObjectPool>());
+    private final Map<Class<?>, GenericObjectPool> dePools = Collections
+        .synchronizedMap(new HashMap<Class<?>, GenericObjectPool>());
+    private final Map<Class<?>, GenericObjectPool> cPools = Collections
+        .synchronizedMap(new HashMap<Class<?>, GenericObjectPool>());
+
+    private DirectCodecPool() {}
+
+    public class CodecPool {
+      private final GenericObjectPool compressorPool;
+      private final GenericObjectPool decompressorPool;
+      private final GenericObjectPool directDecompressorPool;
+      private final boolean supportDirectDecompressor;
+      private static final String BYTE_BUF_IMPL_NOT_FOUND_MSG =
+          "Unable to find ByteBuffer based %s for codec %s, will use a byte array based implementation instead.";
+
+      private CodecPool(final CompressionCodec codec){
+        try {
+          boolean supportDirectDecompressor = codec.getClass() == DIRECT_DECOMPRESSION_CODEC_CLASS;
+          compressorPool = new GenericObjectPool(new BasePoolableObjectFactory() {
+            public Object makeObject() throws Exception {
+              return codec.createCompressor();
+            }
+          }, Integer.MAX_VALUE);
+
+          Object com = compressorPool.borrowObject();
+          if (com != null) {
+            cPools.put(com.getClass(), compressorPool);
+            compressorPool.returnObject(com);
+          } else {
+            if (Log.DEBUG) {
+              LOG.debug(String.format(BYTE_BUF_IMPL_NOT_FOUND_MSG, "compressor", codec.getClass().getName()));
+            }
+          }
+
+          decompressorPool = new GenericObjectPool(new BasePoolableObjectFactory() {
+            public Object makeObject() throws Exception {
+              return codec.createDecompressor();
+            }
+          }, Integer.MAX_VALUE);
+
+          Object decom = decompressorPool.borrowObject();
+          if (decom != null) {
+            dePools.put(decom.getClass(), decompressorPool);
+            decompressorPool.returnObject(decom);
+          } else {
+            if (Log.DEBUG) {
+              LOG.debug(String.format(BYTE_BUF_IMPL_NOT_FOUND_MSG, "decompressor" + codec.getClass().getName()));
+            }
+          }
+
+          if (supportDirectDecompressor) {
+            directDecompressorPool = new GenericObjectPool(
+                new BasePoolableObjectFactory() {
+                  public Object makeObject() throws Exception {
+                    return CREATE_DIRECT_DECOMPRESSOR_METHOD.invoke(DIRECT_DECOMPRESSION_CODEC_CLASS);
+                  }
+                }, Integer.MAX_VALUE);
+
+            Object ddecom = directDecompressorPool.borrowObject();
+            if (ddecom != null) {
+              directDePools.put(ddecom.getClass(), directDecompressorPool);
+              directDecompressorPool.returnObject(ddecom);
+
+            } else {
+              supportDirectDecompressor = false;
+              if (Log.DEBUG) {
+                LOG.debug(String.format(BYTE_BUF_IMPL_NOT_FOUND_MSG, "compressor" + codec.getClass().getName()));
+              }
+            }
+
+          } else {
+            directDecompressorPool = null;
+          }
+
+          this.supportDirectDecompressor = supportDirectDecompressor;
+        } catch (Exception e) {
+          throw new ParquetCompressionCodecException("Error creating compression codec pool.", e);
+        }
+      }
+
+      public Object borrowDirectDecompressor(){
+        Preconditions.checkArgument(supportDirectDecompressor, "Tried to get a direct Decompressor from a non-direct codec.");
+        try {
+          return directDecompressorPool.borrowObject();
+        } catch (Exception e) {
+          throw new ParquetCompressionCodecException(e);
+        }
+      }
+
+      public boolean supportsDirectDecompression() {
+        return supportDirectDecompressor;
+      }
+
+      public Decompressor borrowDecompressor(){
+        return borrow(decompressorPool);
+      }
+
+      public Compressor borrowCompressor(){
+        return borrow(compressorPool);
+      }
+    }
+
+    public CodecPool codec(CompressionCodec codec){
+      CodecPool pools = codecs.get(codec);
+      if(pools == null){
+        synchronized(this){
+          pools = codecs.get(codec);
+          if(pools == null){
+            pools = new CodecPool(codec);
+            codecs.put(codec, pools);
+          }
+        }
+      }
+      return pools;
+    }
+
+    private void returnToPool(Object obj, Map<Class<?>, GenericObjectPool> pools) {
+      try {
+        GenericObjectPool pool = pools.get(obj.getClass());
+        if (pool == null) {
+          throw new IllegalStateException("Received unexpected compressor or decompressor, " +
+              "cannot be returned to any available pool: " + obj.getClass().getSimpleName());
+        }
+        pool.returnObject(obj);
+      } catch (Exception e) {
+        throw new ParquetCompressionCodecException(e);
+      }
+    }
+
+    /**
+     * Borrow an object from a pool.
+     *
+     * @param pool - the pull to borrow from, must not be null
+     * @return - an object from the pool
+     */
+    @SuppressWarnings("unchecked")
+    public <T> T borrow(GenericObjectPool pool) {
+      try {
+        return (T) pool.borrowObject();
+      } catch (Exception e) {
+        throw new ParquetCompressionCodecException(e);
+      }
+
+    }
+
+    public void returnCompressor(Compressor compressor) {
+      returnToPool(compressor, cPools);
+    }
+
+    public void returnDecompressor(Decompressor decompressor) {
+      returnToPool(decompressor, dePools);
+    }
+
+    public void returnDirectDecompressor(Object decompressor) {
+      returnToPool(decompressor, directDePools);
+    }
+
+    public static class ParquetCompressionCodecException extends ParquetRuntimeException {
+
+      public ParquetCompressionCodecException() {
+        super();
+      }
+
+      public ParquetCompressionCodecException(String message, Throwable cause) {
+        super(message, cause);
+      }
+
+      public ParquetCompressionCodecException(String message) {
+        super(message);
+      }
+
+      public ParquetCompressionCodecException(Throwable cause) {
+
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java
index ab9cb3e..87b23a2 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *   http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@@ -28,6 +28,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.column.ColumnWriteStore;
 import org.apache.parquet.column.ParquetProperties;
@@ -86,7 +87,8 @@ class InternalParquetRecordWriter<T> {
       int dictionaryPageSize,
       boolean enableDictionary,
       boolean validating,
-      WriterVersion writerVersion) {
+      WriterVersion writerVersion,
+      ByteBufferAllocator allocator) {
     this.parquetFileWriter = parquetFileWriter;
     this.writeSupport = checkNotNull(writeSupport, "writeSupport");
     this.schema = schema;
@@ -97,16 +99,17 @@ class InternalParquetRecordWriter<T> {
     this.pageSize = pageSize;
     this.compressor = compressor;
     this.validating = validating;
-    this.parquetProperties = new ParquetProperties(dictionaryPageSize, writerVersion, enableDictionary);
+    this.parquetProperties = new ParquetProperties(dictionaryPageSize, writerVersion, enableDictionary, allocator);
     initStore();
   }
 
   private void initStore() {
-    pageStore = new ColumnChunkPageWriteStore(compressor, schema, pageSize);
+    pageStore = new ColumnChunkPageWriteStore(compressor, schema, parquetProperties.getAllocator());
     columnStore = parquetProperties.newColumnWriteStore(
         schema,
         pageStore,
-        pageSize);
+        pageSize,
+        parquetProperties.getAllocator());
     MessageColumnIO columnIO = new ColumnIOFactory(validating).getColumnIO(schema);
     this.recordConsumer = columnIO.getRecordWriter(columnStore);
     writeSupport.prepareForWrite(recordConsumer);
@@ -150,7 +153,9 @@ class InternalParquetRecordWriter<T> {
             max(MINIMUM_RECORD_COUNT_FOR_CHECK, (recordCount + (long)(nextRowGroupSize / ((float)recordSize))) / 2), // will check halfway
             recordCount + MAXIMUM_RECORD_COUNT_FOR_CHECK // will not look more than max records ahead
             );
-        if (DEBUG) LOG.debug(format("Checked mem at %,d will check again at: %,d ", recordCount, recordCountForNextMemCheck));
+        if (DEBUG) {
+          LOG.debug(format("Checked mem at %,d will check again at: %,d ", recordCount, recordCountForNextMemCheck));
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
index f43e692..c54b2b2 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *   http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@@ -27,10 +27,10 @@ import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC;
 import static org.apache.parquet.hadoop.ParquetFileWriter.PARQUET_COMMON_METADATA_FILE;
 import static org.apache.parquet.hadoop.ParquetFileWriter.PARQUET_METADATA_FILE;
 
-import java.io.ByteArrayInputStream;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.SequenceInputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -53,6 +53,11 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
+import org.apache.parquet.hadoop.util.CompatibilityUtil;
+
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.ColumnDescriptor;
@@ -185,7 +190,9 @@ public class ParquetFileReader implements Closeable {
 
     if (toRead.size() > 0) {
       // read the footers of the files that did not have a summary file
-      if (Log.INFO) LOG.info("reading another " + toRead.size() + " footers");
+      if (Log.INFO) {
+        LOG.info("reading another " + toRead.size() + " footers");
+      }
       result.addAll(readAllFootersInParallel(configuration, toRead, skipRowGroups));
     }
 
@@ -297,7 +304,7 @@ public class ParquetFileReader implements Closeable {
    * Read the footers of all the files under that path (recursively)
    * using summary files if possible
    * @param configuration the configuration to access the FS
-   * @param fileStatus the root dir
+   * @param pathStatus the root dir
    * @return all the footers
    * @throws IOException
    */
@@ -342,7 +349,9 @@ public class ParquetFileReader implements Closeable {
       if (Log.INFO) LOG.info("reading summary file: " + commonMetaDataFile);
       return readFooter(configuration, commonMetaDataFile, filter(skipRowGroups));
     } else if (fileSystem.exists(metadataFile)) {
-      if (Log.INFO) LOG.info("reading summary file: " + metadataFile);
+      if (Log.INFO) {
+        LOG.info("reading summary file: " + metadataFile);
+      }
       return readFooter(configuration, metadataFile, filter(skipRowGroups));
     } else {
       return null;
@@ -416,13 +425,17 @@ public class ParquetFileReader implements Closeable {
     FSDataInputStream f = fileSystem.open(file.getPath());
     try {
       long l = file.getLen();
-      if (Log.DEBUG) LOG.debug("File length " + l);
+      if (Log.DEBUG) {
+        LOG.debug("File length " + l);
+      }
       int FOOTER_LENGTH_SIZE = 4;
       if (l < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + footerIndex + MAGIC
         throw new RuntimeException(file.getPath() + " is not a Parquet file (too small)");
       }
       long footerLengthIndex = l - FOOTER_LENGTH_SIZE - MAGIC.length;
-      if (Log.DEBUG) LOG.debug("reading footer index at " + footerLengthIndex);
+      if (Log.DEBUG) {
+        LOG.debug("reading footer index at " + footerLengthIndex);
+      }
 
       f.seek(footerLengthIndex);
       int footerLength = readIntLittleEndian(f);
@@ -432,7 +445,9 @@ public class ParquetFileReader implements Closeable {
         throw new RuntimeException(file.getPath() + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) + " but found " + Arrays.toString(magic));
       }
       long footerIndex = footerLengthIndex - footerLength;
-      if (Log.DEBUG) LOG.debug("read footer length: " + footerLength + ", footer index: " + footerIndex);
+      if (Log.DEBUG) {
+        LOG.debug("read footer length: " + footerLength + ", footer index: " + footerIndex);
+      }
       if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) {
         throw new RuntimeException("corrupted file: the footer index is not within the file");
       }
@@ -450,6 +465,7 @@ public class ParquetFileReader implements Closeable {
   private final Map<ColumnPath, ColumnDescriptor> paths = new HashMap<ColumnPath, ColumnDescriptor>();
   private final FileMetaData fileMetaData;
   private final String createdBy;
+  private final ByteBufferAllocator allocator;
 
   private int currentBlock = 0;
 
@@ -480,7 +496,10 @@ public class ParquetFileReader implements Closeable {
     for (ColumnDescriptor col : columns) {
       paths.put(ColumnPath.get(col.getPath()), col);
     }
-    this.codecFactory = new CodecFactory(configuration);
+    // the page size parameter isn't meaningful when only using
+    // the codec factory to get decompressors
+    this.codecFactory = new CodecFactory(configuration, 0);
+    this.allocator = new HeapByteBufferAllocator();
   }
 
 
@@ -540,7 +559,7 @@ public class ParquetFileReader implements Closeable {
    * @author Julien Le Dem
    *
    */
-  private class Chunk extends ByteArrayInputStream {
+  private class Chunk extends ByteBufferInputStream {
 
     private final ChunkDescriptor descriptor;
 
@@ -550,10 +569,9 @@ public class ParquetFileReader implements Closeable {
      * @param data contains the chunk data at offset
      * @param offset where the chunk starts in offset
      */
-    public Chunk(ChunkDescriptor descriptor, byte[] data, int offset) {
-      super(data);
+    public Chunk(ChunkDescriptor descriptor, ByteBuffer data, int offset) {
+      super(data, offset, descriptor.size);
       this.descriptor = descriptor;
-      this.pos = offset;
     }
 
     protected PageHeader readPageHeader() throws IOException {
@@ -626,7 +644,9 @@ public class ParquetFileReader implements Closeable {
             valuesCountReadSoFar += dataHeaderV2.getNum_values();
             break;
           default:
-            if (DEBUG) LOG.debug("skipping page of type " + pageHeader.getType() + " of size " + compressedPageSize);
+            if (DEBUG) {
+              LOG.debug("skipping page of type " + pageHeader.getType() + " of size " + compressedPageSize);
+            }
             this.skip(compressedPageSize);
             break;
         }
@@ -647,7 +667,7 @@ public class ParquetFileReader implements Closeable {
      * @return the current position in the chunk
      */
     public int pos() {
-      return this.pos;
+      return this.byteBuf.position();
     }
 
     /**
@@ -656,8 +676,9 @@ public class ParquetFileReader implements Closeable {
      * @throws IOException
      */
     public BytesInput readAsBytesInput(int size) throws IOException {
-      final BytesInput r = BytesInput.from(this.buf, this.pos, size);
-      this.pos += size;
+      int pos = this.byteBuf.position();
+      final BytesInput r = BytesInput.from(this.byteBuf, pos, size);
+      this.byteBuf.position(pos + size);
       return r;
     }
 
@@ -675,18 +696,18 @@ public class ParquetFileReader implements Closeable {
 
     /**
      * @param descriptor the descriptor of the chunk
-     * @param data contains the data of the chunk at offset
+     * @param byteBuf contains the data of the chunk at offset
      * @param offset where the chunk starts in data
      * @param f the file stream positioned at the end of this chunk
      */
-    private WorkaroundChunk(ChunkDescriptor descriptor, byte[] data, int offset, FSDataInputStream f) {
-      super(descriptor, data, offset);
+    private WorkaroundChunk(ChunkDescriptor descriptor, ByteBuffer byteBuf, int offset, FSDataInputStream f) {
+      super(descriptor, byteBuf, offset);
       this.f = f;
     }
 
     protected PageHeader readPageHeader() throws IOException {
       PageHeader pageHeader;
-      int initialPos = this.pos;
+      int initialPos = pos();
       try {
         pageHeader = Util.readPageHeader(this);
       } catch (IOException e) {
@@ -695,7 +716,7 @@ public class ParquetFileReader implements Closeable {
         // to allow reading older files (using dictionary) we need this.
         // usually 13 to 19 bytes are missing
         // if the last page is smaller than this, the page header itself is truncated in the buffer.
-        this.pos = initialPos; // resetting the buffer to the position before we got the error
+        this.byteBuf.rewind(); // resetting the buffer to the position before we got the error
         LOG.info("completing the column chunk to read the page header");
         pageHeader = Util.readPageHeader(new SequenceInputStream(this, f)); // trying again from the buffer + remainder of the stream.
       }
@@ -703,12 +724,12 @@ public class ParquetFileReader implements Closeable {
     }
 
     public BytesInput readAsBytesInput(int size) throws IOException {
-      if (pos + size > count) {
+      if (pos() + size > initPos + count) {
         // this is to workaround a bug where the compressedLength
         // of the chunk is missing the size of the header of the dictionary
         // to allow reading older files (using dictionary) we need this.
         // usually 13 to 19 bytes are missing
-        int l1 = count - pos;
+        int l1 = initPos + count - pos();
         int l2 = size - l1;
         LOG.info("completed the column chunk with " + l2 + " bytes");
         return BytesInput.concat(super.readAsBytesInput(l1), BytesInput.copy(BytesInput.from(f, l2)));
@@ -784,18 +805,18 @@ public class ParquetFileReader implements Closeable {
     public List<Chunk> readAll(FSDataInputStream f) throws IOException {
       List<Chunk> result = new ArrayList<Chunk>(chunks.size());
       f.seek(offset);
-      byte[] chunksBytes = new byte[length];
-      f.readFully(chunksBytes);
+      ByteBuffer chunksByteBuffer = allocator.allocate(length);
+      CompatibilityUtil.getBuf(f, chunksByteBuffer, length);
       // report in a counter the data we just scanned
       BenchmarkCounter.incrementBytesRead(length);
       int currentChunkOffset = 0;
       for (int i = 0; i < chunks.size(); i++) {
         ChunkDescriptor descriptor = chunks.get(i);
         if (i < chunks.size() - 1) {
-          result.add(new Chunk(descriptor, chunksBytes, currentChunkOffset));
+          result.add(new Chunk(descriptor, chunksByteBuffer, currentChunkOffset));
         } else {
           // because of a bug, the last chunk might be larger than descriptor.size
-          result.add(new WorkaroundChunk(descriptor, chunksBytes, currentChunkOffset, f));
+          result.add(new WorkaroundChunk(descriptor, chunksByteBuffer, currentChunkOffset, f));
         }
         currentChunkOffset += descriptor.size;
       }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
index 664ee9d..8683a18 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
@@ -73,8 +73,9 @@ public class ParquetFileWriter {
   private static ParquetMetadataConverter metadataConverter = new ParquetMetadataConverter();
 
   public static final String PARQUET_METADATA_FILE = "_metadata";
+  public static final String MAGIC_STR = "PAR1";
+  public static final byte[] MAGIC = MAGIC_STR.getBytes(Charset.forName("ASCII"));
   public static final String PARQUET_COMMON_METADATA_FILE = "_common_metadata";
-  public static final byte[] MAGIC = "PAR1".getBytes(Charset.forName("ASCII"));
   public static final int CURRENT_VERSION = 1;
 
   // need to supply a buffer size when setting block size. this is the default

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
index ad6c034..562bffc 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
@@ -341,7 +341,6 @@ public class ParquetOutputFormat<T> extends FileOutputFormat<Void, T> {
         throws IOException, InterruptedException {
     final WriteSupport<T> writeSupport = getWriteSupport(conf);
 
-    CodecFactory codecFactory = new CodecFactory(conf);
     long blockSize = getLongBlockSize(conf);
     if (INFO) LOG.info("Parquet block size to " + blockSize);
     int pageSize = getPageSize(conf);
@@ -357,6 +356,8 @@ public class ParquetOutputFormat<T> extends FileOutputFormat<Void, T> {
     int maxPaddingSize = getMaxPaddingSize(conf);
     if (INFO) LOG.info("Maximum row group padding size is " + maxPaddingSize + " bytes");
 
+    CodecFactory codecFactory = new CodecFactory(conf, pageSize);
+
     WriteContext init = writeSupport.init(conf);
     ParquetFileWriter w = new ParquetFileWriter(
         conf, init.getSchema(), file, Mode.CREATE, blockSize, maxPaddingSize);
@@ -379,7 +380,7 @@ public class ParquetOutputFormat<T> extends FileOutputFormat<Void, T> {
         init.getSchema(),
         init.getExtraMetaData(),
         blockSize, pageSize,
-        codecFactory.getCompressor(codec, pageSize),
+        codecFactory.getCompressor(codec),
         dictionaryPageSize,
         enableDictionary,
         validating,

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordWriter.java
index 2449192..eefb257 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetRecordWriter.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *   http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@@ -20,9 +20,11 @@ package org.apache.parquet.hadoop;
 
 import java.io.IOException;
 import java.util.Map;
+
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import org.apache.parquet.column.ParquetProperties.WriterVersion;
 import org.apache.parquet.hadoop.CodecFactory.BytesCompressor;
 import org.apache.parquet.hadoop.api.WriteSupport;
@@ -70,7 +72,7 @@ public class ParquetRecordWriter<T> extends RecordWriter<Void, T> {
       WriterVersion writerVersion) {
     internalWriter = new InternalParquetRecordWriter<T>(w, writeSupport, schema,
         extraMetaData, blockSize, pageSize, compressor, dictionaryPageSize, enableDictionary,
-        validating, writerVersion);
+      validating, writerVersion, new HeapByteBufferAllocator());
   }
 
   /**
@@ -98,8 +100,8 @@ public class ParquetRecordWriter<T> extends RecordWriter<Void, T> {
       WriterVersion writerVersion,
       MemoryManager memoryManager) {
     internalWriter = new InternalParquetRecordWriter<T>(w, writeSupport, schema,
-        extraMetaData, blockSize, pageSize, compressor, dictionaryPageSize, enableDictionary,
-        validating, writerVersion);
+      extraMetaData, blockSize, pageSize, compressor, dictionaryPageSize, enableDictionary,
+      validating, writerVersion, new HeapByteBufferAllocator());
     this.memoryManager = checkNotNull(memoryManager, "memoryManager");
     memoryManager.addWriter(internalWriter, blockSize);
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
index e3b7953..e2521fb 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
@@ -29,6 +29,7 @@ import org.apache.parquet.column.ParquetProperties.WriterVersion;
 import org.apache.parquet.hadoop.api.WriteSupport;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 
 /**
  * Write records to a Parquet file.
@@ -267,8 +268,8 @@ public class ParquetWriter<T> implements Closeable {
         conf, schema, file, mode, blockSize, maxPaddingSize);
     fileWriter.start();
 
-    CodecFactory codecFactory = new CodecFactory(conf);
-    CodecFactory.BytesCompressor compressor =	codecFactory.getCompressor(compressionCodecName, 0);
+    CodecFactory codecFactory = new CodecFactory(conf, pageSize);
+    CodecFactory.BytesCompressor compressor =	codecFactory.getCompressor(compressionCodecName);
     this.writer = new InternalParquetRecordWriter<T>(
         fileWriter,
         writeSupport,
@@ -280,7 +281,8 @@ public class ParquetWriter<T> implements Closeable {
         dictionaryPageSize,
         enableDictionary,
         validating,
-        writerVersion);
+        writerVersion,
+        new HeapByteBufferAllocator());
   }
 
   public void write(T object) throws IOException {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java
index 8631267..66e3b81 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java
@@ -147,4 +147,5 @@ public class SnappyDecompressor implements Decompressor {
   public void setDictionary(byte[] b, int off, int len) {
     // No-op		
   }
-}
+
+} //class SnappyDecompressor

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/CompatibilityUtil.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/CompatibilityUtil.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/CompatibilityUtil.java
new file mode 100644
index 0000000..bacf222
--- /dev/null
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/CompatibilityUtil.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.hadoop.util;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.parquet.ShouldNeverHappenException;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+public class CompatibilityUtil {
+
+  // Will be set to true if the implementation of FSDataInputSteam supports
+  // the 2.x APIs, in particular reading using a provided ByteBuffer
+  private static boolean useV21;
+  public static final V21FileAPI fileAPI;
+
+  private static class V21FileAPI {
+    private final Method PROVIDE_BUF_READ_METHOD;
+    private final Class<?> FSDataInputStreamCls;
+
+    private V21FileAPI() throws ReflectiveOperationException {
+      final String PACKAGE = "org.apache.hadoop";
+      FSDataInputStreamCls = Class.forName(PACKAGE + ".fs.FSDataInputStream");
+      PROVIDE_BUF_READ_METHOD = FSDataInputStreamCls.getMethod("read", ByteBuffer.class);
+    }
+  }
+  
+  static {
+    // Test to see if a class from the Hadoop 2.x API is available
+    boolean v21 = true;
+    try {
+      Class.forName("org.apache.hadoop.io.compress.DirectDecompressor");
+    } catch (ClassNotFoundException cnfe) {
+      v21 = false;
+    }
+
+    useV21 = v21;
+    try {
+      if (v21) {
+        fileAPI = new V21FileAPI();
+      } else {
+        fileAPI = null;
+      }
+
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalArgumentException("Error finding appropriate interfaces using reflection.", e);
+    }
+  }
+
+  private static Object invoke(Method method, String errorMsg, Object instance, Object... args) {
+    try {
+      return method.invoke(instance, args);
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException(errorMsg, e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalArgumentException(errorMsg, e);
+    }
+  }
+
+  public static int getBuf(FSDataInputStream f, ByteBuffer readBuf, int maxSize) throws IOException {
+    int res;
+    if (useV21) {
+      try {
+        res = (Integer) fileAPI.PROVIDE_BUF_READ_METHOD.invoke(f, readBuf);
+      } catch (InvocationTargetException e) {
+        if (e.getCause() instanceof UnsupportedOperationException) {
+          // the FSDataInputStream docs say specifically that implementations
+          // can choose to throw UnsupportedOperationException, so this should
+          // be a reasonable check to make to see if the interface is
+          // present but not implemented and we should be falling back
+          useV21 = false;
+          return getBuf(f, readBuf, maxSize);
+        } else if (e.getCause() instanceof IOException) {
+          throw (IOException) e.getCause();
+        } else {
+          // To handle any cases where a Runtime exception occurs and provide
+          // some additional context information. A stacktrace would just give
+          // a line number, this at least tells them we were using the version
+          // of the read method designed for using a ByteBuffer.
+          throw new IOException("Error reading out of an FSDataInputStream " +
+              "using the Hadoop 2 ByteBuffer based read method.", e.getCause());
+        }
+      } catch (IllegalAccessException e) {
+        // This method is public because it is defined in an interface,
+        // there should be no problems accessing it
+        throw new ShouldNeverHappenException(e);
+      }
+    } else {
+      byte[] buf = new byte[maxSize];
+      res = f.read(buf);
+      readBuf.put(buf, 0, res);
+    }
+    return res;
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
index 2c644b6..87574cd 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java
@@ -61,6 +61,7 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.MessageTypeParser;
 import org.apache.parquet.schema.Types;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 
 public class TestColumnChunkPageWriteStore {
 
@@ -101,7 +102,7 @@ public class TestColumnChunkPageWriteStore {
       writer.start();
       writer.startBlock(rowCount);
       {
-        ColumnChunkPageWriteStore store = new ColumnChunkPageWriteStore(compressor(GZIP), schema , initialSize);
+        ColumnChunkPageWriteStore store = new ColumnChunkPageWriteStore(compressor(GZIP), schema , new HeapByteBufferAllocator());
         PageWriter pageWriter = store.getPageWriter(col);
         pageWriter.writePageV2(
             rowCount, nullCount, valueCount,
@@ -158,8 +159,10 @@ public class TestColumnChunkPageWriteStore {
     int fakeCount = 3;
     BinaryStatistics fakeStats = new BinaryStatistics();
 
+    // TODO - look back at this, an allocator was being passed here in the ByteBuffer changes
+    // see comment at this constructor
     ColumnChunkPageWriteStore store = new ColumnChunkPageWriteStore(
-        compressor(UNCOMPRESSED), schema, initialSize);
+        compressor(UNCOMPRESSED), schema, new HeapByteBufferAllocator());
 
     for (ColumnDescriptor col : schema.getColumns()) {
       PageWriter pageWriter = store.getPageWriter(col);
@@ -176,6 +179,6 @@ public class TestColumnChunkPageWriteStore {
   }
 
   private CodecFactory.BytesCompressor compressor(CompressionCodecName codec) {
-    return new CodecFactory(conf).getCompressor(codec, pageSize);
+    return new CodecFactory(conf, pageSize).getCompressor(codec);
   }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDirectCodecFactory.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDirectCodecFactory.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDirectCodecFactory.java
new file mode 100644
index 0000000..caf2ed6
--- /dev/null
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDirectCodecFactory.java
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.parquet.hadoop;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.bytes.ByteBufferAllocator;
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+public class TestDirectCodecFactory {
+
+  private static enum Decompression {
+    ON_HEAP, OFF_HEAP, OFF_HEAP_BYTES_INPUT
+  }
+
+  private final int pageSize = 64 * 1024;
+
+  private void test(int size, CompressionCodecName codec, boolean useOnHeapCompression, Decompression decomp) {
+    ByteBuffer rawBuf = null;
+    ByteBuffer outBuf = null;
+    ByteBufferAllocator allocator = null;
+    try {
+      allocator = new DirectByteBufferAllocator();
+      final CodecFactory codecFactory = CodecFactory.createDirectCodecFactory(new Configuration(), allocator, pageSize);
+      rawBuf = allocator.allocate(size);
+      final byte[] rawArr = new byte[size];
+      outBuf = allocator.allocate(size * 2);
+      final Random r = new Random();
+      final byte[] random = new byte[1024];
+      int pos = 0;
+      while (pos < size) {
+        r.nextBytes(random);
+        rawBuf.put(random);
+        System.arraycopy(random, 0, rawArr, pos, random.length);
+        pos += random.length;
+      }
+      rawBuf.flip();
+
+      final DirectCodecFactory.BytesCompressor c = codecFactory.getCompressor(codec);
+      final CodecFactory.BytesDecompressor d = codecFactory.getDecompressor(codec);
+
+      final BytesInput compressed;
+      if (useOnHeapCompression) {
+        compressed = c.compress(BytesInput.from(rawArr));
+      } else {
+        compressed = c.compress(BytesInput.from(rawBuf, 0, rawBuf.remaining()));
+      }
+
+      switch (decomp) {
+        case OFF_HEAP: {
+          final ByteBuffer buf = compressed.toByteBuffer();
+          final ByteBuffer b = allocator.allocate(buf.capacity());
+          try {
+            b.put(buf);
+            b.flip();
+            d.decompress(b, (int) compressed.size(), outBuf, size);
+            for (int i = 0; i < size; i++) {
+              Assert.assertTrue("Data didn't match at " + i, outBuf.get(i) == rawBuf.get(i));
+            }
+          } finally {
+            allocator.release(b);
+          }
+          break;
+        }
+
+        case OFF_HEAP_BYTES_INPUT: {
+          final ByteBuffer buf = compressed.toByteBuffer();
+          final ByteBuffer b = allocator.allocate(buf.capacity());
+          try {
+            b.put(buf);
+            b.flip();
+            final BytesInput input = d.decompress(BytesInput.from(b, 0, b.capacity()), size);
+            Assert.assertArrayEquals(
+                String.format("While testing codec %s", codec),
+                input.toByteArray(), rawArr);
+          } finally {
+            allocator.release(b);
+          }
+          break;
+        }
+        case ON_HEAP: {
+          final byte[] buf = compressed.toByteArray();
+          final BytesInput input = d.decompress(BytesInput.from(buf), size);
+          Assert.assertArrayEquals(input.toByteArray(), rawArr);
+          break;
+        }
+      }
+    } catch (Exception e) {
+      final String msg = String.format(
+          "Failure while testing Codec: %s, OnHeapCompressionInput: %s, Decompression Mode: %s, Data Size: %d",
+          codec.name(),
+          useOnHeapCompression, decomp.name(), size);
+      System.out.println(msg);
+      throw new RuntimeException(msg, e);
+    } finally {
+      if (rawBuf != null) {
+        allocator.release(rawBuf);
+      }
+      if (outBuf != null) {
+        allocator.release(rawBuf);
+      }
+    }
+  }
+
+  @Test
+  public void createDirectFactoryWithHeapAllocatorFails() {
+    String errorMsg = "Test failed, creation of a direct codec factory should have failed when passed a non-direct allocator.";
+    try {
+      CodecFactory.createDirectCodecFactory(new Configuration(), new HeapByteBufferAllocator(), 0);
+      throw new RuntimeException(errorMsg);
+    } catch (IllegalStateException ex) {
+      // indicates successful completion of the test
+      Assert.assertTrue("Missing expected error message.",
+          ex.getMessage()
+          .contains("A DirectCodecFactory requires a direct buffer allocator be provided.")
+      );
+    } catch (Exception ex) {
+      throw new RuntimeException(errorMsg + " Failed with the wrong error.");
+    }
+  }
+
+  @Test
+  public void compressionCodecs() throws Exception {
+    final int[] sizes = { 4 * 1024, 1 * 1024 * 1024 };
+    final boolean[] comp = { true, false };
+
+    for (final int size : sizes) {
+      for (final boolean useOnHeapComp : comp) {
+        for (final Decompression decomp : Decompression.values()) {
+          for (final CompressionCodecName codec : CompressionCodecName.values()) {
+            if (codec == CompressionCodecName.LZO) {
+              // not installed as gpl.
+              continue;
+            }
+            test(size, codec, useOnHeapComp, decomp);
+          }
+        }
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-pig/src/test/java/org/apache/parquet/pig/TupleConsumerPerfTest.java
----------------------------------------------------------------------
diff --git a/parquet-pig/src/test/java/org/apache/parquet/pig/TupleConsumerPerfTest.java b/parquet-pig/src/test/java/org/apache/parquet/pig/TupleConsumerPerfTest.java
index 5e1f5af..c050922 100644
--- a/parquet-pig/src/test/java/org/apache/parquet/pig/TupleConsumerPerfTest.java
+++ b/parquet-pig/src/test/java/org/apache/parquet/pig/TupleConsumerPerfTest.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.logging.Level;
 
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.NonSpillableDataBag;
@@ -59,7 +60,7 @@ public class TupleConsumerPerfTest {
     MessageType schema = new PigSchemaConverter().convert(Utils.getSchemaFromString(pigSchema));
 
     MemPageStore memPageStore = new MemPageStore(0);
-    ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 50*1024*1024, 50*1024*1024, false, WriterVersion.PARQUET_1_0);
+    ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 50*1024*1024, 50*1024*1024, false, WriterVersion.PARQUET_1_0, new HeapByteBufferAllocator());
     write(memPageStore, columns, schema, pigSchema);
     columns.flush();
     read(memPageStore, pigSchema, pigSchemaProjected, pigSchemaNoString);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetReadProtocol.java
----------------------------------------------------------------------
diff --git a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetReadProtocol.java b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetReadProtocol.java
index f5f3ff1..f954e4c 100644
--- a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetReadProtocol.java
+++ b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetReadProtocol.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import thrift.test.OneOfEach;
 
 import org.apache.thrift.TBase;
@@ -148,7 +149,8 @@ public class TestParquetReadProtocol {
     final MessageType schema = schemaConverter.convert(thriftClass);
     LOG.info(schema);
     final MessageColumnIO columnIO = new ColumnIOFactory(true).getColumnIO(schema);
-    final ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 10000, 10000, false, WriterVersion.PARQUET_1_0);
+    final ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 10000, 10000, false,
+        WriterVersion.PARQUET_1_0, new HeapByteBufferAllocator());
     final RecordConsumer recordWriter = columnIO.getRecordWriter(columns);
     final StructType thriftType = schemaConverter.toStructType(thriftClass);
     ParquetWriteProtocol parquetWriteProtocol = new ParquetWriteProtocol(recordWriter, columnIO, thriftType);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dc27f4c..14feb9c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -88,7 +88,7 @@
     <scala.binary.version>2.10</scala.binary.version>
     <scala.maven.test.skip>false</scala.maven.test.skip>
     <pig.version>0.11.1</pig.version>
-    <pig.classifier />
+    <pig.classifier/>
     <thrift.version>0.7.0</thrift.version>
     <fastutil.version>6.5.7</fastutil.version>
     <semver.api.version>0.9.33</semver.api.version>
@@ -225,6 +225,7 @@
                      <exclude>org/apache/parquet/filter2/**</exclude>
                      <exclude>org/apache/parquet/column/**</exclude>
                      <exclude>org/apache/parquet/hadoop/ParquetInputSplit</exclude>
+                     <exclude>org/apache/parquet/hadoop/CodecFactory**</exclude>
                      <exclude>shaded/**</exclude> <!-- shaded by parquet -->
                      <!-- temporary exclusions for false-positives -->
                      <exclude>org/apache/parquet/Version</exclude>


[3/4] parquet-mr git commit: PARQUET-77: ByteBuffer use in read and write paths

Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java
index eb9fdd9..86edd79 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java
@@ -36,9 +36,11 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap;
 import it.unimi.dsi.fastutil.objects.ObjectIterator;
 
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.BytesUtils;
@@ -92,17 +94,28 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
   /* dictionary encoded values */
   protected IntList encodedValues = new IntList();
 
+  /** indicates if this is the first page being processed */
+  protected boolean firstPage = true;
+
+  protected ByteBufferAllocator allocator;
+  /* Track the list of writers used so they can be appropriately closed when necessary
+     (currently used for off-heap memory which is not garbage collected) */
+  private List<RunLengthBitPackingHybridEncoder> encoders = new ArrayList<RunLengthBitPackingHybridEncoder>();
+
   /**
    * @param maxDictionaryByteSize
    */
-  protected DictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage) {
+  protected DictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage, ByteBufferAllocator allocator) {
+    this.allocator = allocator;
     this.maxDictionaryByteSize = maxDictionaryByteSize;
     this.encodingForDataPage = encodingForDataPage;
     this.encodingForDictionaryPage = encodingForDictionaryPage;
   }
 
-  protected DictionaryPage dictPage(ValuesWriter dictionaryEncoder) {
-    return new DictionaryPage(dictionaryEncoder.getBytes(), lastUsedDictionarySize, encodingForDictionaryPage);
+  protected DictionaryPage dictPage(ValuesWriter dictPageWriter) {
+    DictionaryPage ret = new DictionaryPage(dictPageWriter.getBytes(), lastUsedDictionarySize, encodingForDictionaryPage);
+    dictPageWriter.close();
+    return ret;
   }
 
   @Override
@@ -147,12 +160,12 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     int maxDicId = getDictionarySize() - 1;
     if (DEBUG) LOG.debug("max dic id " + maxDicId);
     int bitWidth = BytesUtils.getWidthFromMaxInt(maxDicId);
-
     int initialSlabSize =
         CapacityByteArrayOutputStream.initialSlabSizeHeuristic(MIN_INITIAL_SLAB_SIZE, maxDictionaryByteSize, 10);
 
     RunLengthBitPackingHybridEncoder encoder =
-        new RunLengthBitPackingHybridEncoder(bitWidth, initialSlabSize, maxDictionaryByteSize);
+        new RunLengthBitPackingHybridEncoder(bitWidth, initialSlabSize, maxDictionaryByteSize, this.allocator);
+    encoders.add(encoder);
     IntIterator iterator = encodedValues.iterator();
     try {
       while (iterator.hasNext()) {
@@ -179,10 +192,20 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
 
   @Override
   public void reset() {
+    close();
     encodedValues = new IntList();
   }
 
   @Override
+  public void close() {
+    encodedValues = null;
+    for (RunLengthBitPackingHybridEncoder encoder : encoders) {
+      encoder.close();
+    }
+    encoders.clear();
+  }
+
+  @Override
   public void resetDictionary() {
     lastUsedDictionaryByteSize = 0;
     lastUsedDictionarySize = 0;
@@ -225,8 +248,8 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     /**
      * @param maxDictionaryByteSize
      */
-    public PlainBinaryDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage) {
-      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage);
+    public PlainBinaryDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage, ByteBufferAllocator allocator) {
+      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage, allocator);
       binaryDictionaryContent.defaultReturnValue(-1);
     }
 
@@ -243,10 +266,10 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     }
 
     @Override
-    public DictionaryPage createDictionaryPage() {
+    public DictionaryPage toDictPageAndClose() {
       if (lastUsedDictionarySize > 0) {
         // return a dictionary only if we actually used it
-        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize);
+        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize, allocator);
         Iterator<Binary> binaryIterator = binaryDictionaryContent.keySet().iterator();
         // write only the part of the dict that we used
         for (int i = 0; i < lastUsedDictionarySize; i++) {
@@ -294,10 +317,9 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
 
     /**
      * @param maxDictionaryByteSize
-     * @param initialSize
      */
-    public PlainFixedLenArrayDictionaryValuesWriter(int maxDictionaryByteSize, int length, Encoding encodingForDataPage, Encoding encodingForDictionaryPage) {
-      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage);
+    public PlainFixedLenArrayDictionaryValuesWriter(int maxDictionaryByteSize, int length, Encoding encodingForDataPage, Encoding encodingForDictionaryPage, ByteBufferAllocator allocator) {
+      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage, allocator);
       this.length = length;
     }
 
@@ -313,10 +335,10 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     }
 
     @Override
-    public DictionaryPage createDictionaryPage() {
+    public DictionaryPage toDictPageAndClose() {
       if (lastUsedDictionarySize > 0) {
         // return a dictionary only if we actually used it
-        FixedLenByteArrayPlainValuesWriter dictionaryEncoder = new FixedLenByteArrayPlainValuesWriter(length, lastUsedDictionaryByteSize, maxDictionaryByteSize);
+        FixedLenByteArrayPlainValuesWriter dictionaryEncoder = new FixedLenByteArrayPlainValuesWriter(length, lastUsedDictionaryByteSize, maxDictionaryByteSize, allocator);
         Iterator<Binary> binaryIterator = binaryDictionaryContent.keySet().iterator();
         // write only the part of the dict that we used
         for (int i = 0; i < lastUsedDictionarySize; i++) {
@@ -339,10 +361,9 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
 
     /**
      * @param maxDictionaryByteSize
-     * @param initialSize
      */
-    public PlainLongDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage) {
-      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage);
+    public PlainLongDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage, ByteBufferAllocator allocator) {
+      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage, allocator);
       longDictionaryContent.defaultReturnValue(-1);
     }
 
@@ -358,10 +379,10 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     }
 
     @Override
-    public DictionaryPage createDictionaryPage() {
+    public DictionaryPage toDictPageAndClose() {
       if (lastUsedDictionarySize > 0) {
         // return a dictionary only if we actually used it
-        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize);
+        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize, allocator);
         LongIterator longIterator = longDictionaryContent.keySet().iterator();
         // write only the part of the dict that we used
         for (int i = 0; i < lastUsedDictionarySize; i++) {
@@ -411,10 +432,9 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
 
     /**
      * @param maxDictionaryByteSize
-     * @param initialSize
      */
-    public PlainDoubleDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage) {
-      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage);
+    public PlainDoubleDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage, ByteBufferAllocator allocator) {
+      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage, allocator);
       doubleDictionaryContent.defaultReturnValue(-1);
     }
 
@@ -430,10 +450,10 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     }
 
     @Override
-    public DictionaryPage createDictionaryPage() {
+    public DictionaryPage toDictPageAndClose() {
       if (lastUsedDictionarySize > 0) {
         // return a dictionary only if we actually used it
-        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize);
+        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize, allocator);
         DoubleIterator doubleIterator = doubleDictionaryContent.keySet().iterator();
         // write only the part of the dict that we used
         for (int i = 0; i < lastUsedDictionarySize; i++) {
@@ -483,10 +503,9 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
 
     /**
      * @param maxDictionaryByteSize
-     * @param initialSize
      */
-    public PlainIntegerDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage) {
-      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage);
+    public PlainIntegerDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage, ByteBufferAllocator allocator) {
+      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage, allocator);
       intDictionaryContent.defaultReturnValue(-1);
     }
 
@@ -502,10 +521,10 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     }
 
     @Override
-    public DictionaryPage createDictionaryPage() {
+    public DictionaryPage toDictPageAndClose() {
       if (lastUsedDictionarySize > 0) {
         // return a dictionary only if we actually used it
-        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize);
+        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize, allocator);
         it.unimi.dsi.fastutil.ints.IntIterator intIterator = intDictionaryContent.keySet().iterator();
         // write only the part of the dict that we used
         for (int i = 0; i < lastUsedDictionarySize; i++) {
@@ -555,10 +574,9 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
 
     /**
      * @param maxDictionaryByteSize
-     * @param initialSize
      */
-    public PlainFloatDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage) {
-      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage);
+    public PlainFloatDictionaryValuesWriter(int maxDictionaryByteSize, Encoding encodingForDataPage, Encoding encodingForDictionaryPage, ByteBufferAllocator allocator) {
+      super(maxDictionaryByteSize, encodingForDataPage, encodingForDictionaryPage, allocator);
       floatDictionaryContent.defaultReturnValue(-1);
     }
 
@@ -574,10 +592,10 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req
     }
 
     @Override
-    public DictionaryPage createDictionaryPage() {
+    public DictionaryPage toDictPageAndClose() {
       if (lastUsedDictionarySize > 0) {
         // return a dictionary only if we actually used it
-        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize);
+        PlainValuesWriter dictionaryEncoder = new PlainValuesWriter(lastUsedDictionaryByteSize, maxDictionaryByteSize, allocator);
         FloatIterator floatIterator = floatDictionaryContent.keySet().iterator();
         // write only the part of the dict that we used
         for (int i = 0; i < lastUsedDictionarySize; i++) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/PlainValuesDictionary.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/PlainValuesDictionary.java b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/PlainValuesDictionary.java
index e671310..0fa6cc6 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/PlainValuesDictionary.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/PlainValuesDictionary.java
@@ -23,6 +23,7 @@ import static org.apache.parquet.column.Encoding.PLAIN_DICTIONARY;
 import static org.apache.parquet.column.Encoding.PLAIN;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.Preconditions;
 import org.apache.parquet.column.Dictionary;
@@ -86,9 +87,10 @@ public abstract class PlainValuesDictionary extends Dictionary {
      */
     public PlainBinaryDictionary(DictionaryPage dictionaryPage, Integer length) throws IOException {
       super(dictionaryPage);
-      final byte[] dictionaryBytes = dictionaryPage.getBytes().toByteArray();
+      final ByteBuffer dictionaryBytes = dictionaryPage.getBytes().toByteBuffer();
       binaryDictionaryContent = new Binary[dictionaryPage.getDictionarySize()];
-      int offset = 0;
+      // dictionary values are stored in order: size (4 bytes LE) followed by {size} bytes
+      int offset = dictionaryBytes.position();
       if (length == null) {
         // dictionary values are stored in order: size (4 bytes LE) followed by {size} bytes
         for (int i = 0; i < binaryDictionaryContent.length; i++) {
@@ -96,7 +98,7 @@ public abstract class PlainValuesDictionary extends Dictionary {
           // read the length
           offset += 4;
           // wrap the content in a binary
-          binaryDictionaryContent[i] = Binary.fromConstantByteArray(dictionaryBytes, offset, len);
+          binaryDictionaryContent[i] = Binary.fromConstantByteBuffer(dictionaryBytes, offset, len);
           // increment to the next value
           offset += len;
         }
@@ -106,7 +108,7 @@ public abstract class PlainValuesDictionary extends Dictionary {
             "Invalid byte array length: " + length);
         for (int i = 0; i < binaryDictionaryContent.length; i++) {
           // wrap the content in a Binary
-          binaryDictionaryContent[i] = Binary.fromConstantByteArray(
+          binaryDictionaryContent[i] = Binary.fromConstantByteBuffer(
               dictionaryBytes, offset, length);
           // increment to the next value
           offset += length;
@@ -148,10 +150,10 @@ public abstract class PlainValuesDictionary extends Dictionary {
      */
     public PlainLongDictionary(DictionaryPage dictionaryPage) throws IOException {
       super(dictionaryPage);
-      final byte[] dictionaryBytes = dictionaryPage.getBytes().toByteArray();
+      final ByteBuffer dictionaryByteBuf = dictionaryPage.getBytes().toByteBuffer();
       longDictionaryContent = new long[dictionaryPage.getDictionarySize()];
       LongPlainValuesReader longReader = new LongPlainValuesReader();
-      longReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryBytes, 0);
+      longReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryByteBuf, dictionaryByteBuf.position());
       for (int i = 0; i < longDictionaryContent.length; i++) {
         longDictionaryContent[i] = longReader.readLong();
       }
@@ -191,10 +193,10 @@ public abstract class PlainValuesDictionary extends Dictionary {
      */
     public PlainDoubleDictionary(DictionaryPage dictionaryPage) throws IOException {
       super(dictionaryPage);
-      final byte[] dictionaryBytes = dictionaryPage.getBytes().toByteArray();
+      final ByteBuffer dictionaryByteBuf = dictionaryPage.getBytes().toByteBuffer();
       doubleDictionaryContent = new double[dictionaryPage.getDictionarySize()];
       DoublePlainValuesReader doubleReader = new DoublePlainValuesReader();
-      doubleReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryBytes, 0);
+      doubleReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryByteBuf, 0);
       for (int i = 0; i < doubleDictionaryContent.length; i++) {
         doubleDictionaryContent[i] = doubleReader.readDouble();
       }
@@ -234,10 +236,10 @@ public abstract class PlainValuesDictionary extends Dictionary {
      */
     public PlainIntegerDictionary(DictionaryPage dictionaryPage) throws IOException {
       super(dictionaryPage);
-      final byte[] dictionaryBytes = dictionaryPage.getBytes().toByteArray();
+      final ByteBuffer dictionaryByteBuf = dictionaryPage.getBytes().toByteBuffer();
       intDictionaryContent = new int[dictionaryPage.getDictionarySize()];
       IntegerPlainValuesReader intReader = new IntegerPlainValuesReader();
-      intReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryBytes, 0);
+      intReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryByteBuf, 0);
       for (int i = 0; i < intDictionaryContent.length; i++) {
         intDictionaryContent[i] = intReader.readInteger();
       }
@@ -277,10 +279,10 @@ public abstract class PlainValuesDictionary extends Dictionary {
      */
     public PlainFloatDictionary(DictionaryPage dictionaryPage) throws IOException {
       super(dictionaryPage);
-      final byte[] dictionaryBytes = dictionaryPage.getBytes().toByteArray();
+      final ByteBuffer dictionaryByteBuf = dictionaryPage.getBytes().toByteBuffer();
       floatDictionaryContent = new float[dictionaryPage.getDictionarySize()];
       FloatPlainValuesReader floatReader = new FloatPlainValuesReader();
-      floatReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryBytes, 0);
+      floatReader.initFromPage(dictionaryPage.getDictionarySize(), dictionaryByteBuf, dictionaryByteBuf.position());
       for (int i = 0; i < floatDictionaryContent.length; i++) {
         floatDictionaryContent[i] = floatReader.readFloat();
       }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/fallback/FallbackValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/fallback/FallbackValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/fallback/FallbackValuesWriter.java
index f66c7c9..19fed7d 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/fallback/FallbackValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/fallback/FallbackValuesWriter.java
@@ -97,11 +97,17 @@ public class FallbackValuesWriter<I extends ValuesWriter & RequiresFallback, F e
     currentWriter.reset();
   }
 
-  public DictionaryPage createDictionaryPage() {
+  @Override
+  public void close() {
+    initialWriter.close();
+    fallBackWriter.close();
+  }
+
+  public DictionaryPage toDictPageAndClose() {
     if (initialUsedAndHadDictionary) {
-      return initialWriter.createDictionaryPage();
+      return initialWriter.toDictPageAndClose();
     } else {
-      return currentWriter.createDictionaryPage();
+      return currentWriter.toDictPageAndClose();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java
index 4346e02..26f5e29 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java
@@ -21,6 +21,7 @@ package org.apache.parquet.column.values.plain;
 import static org.apache.parquet.Log.DEBUG;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesUtils;
@@ -30,7 +31,7 @@ import org.apache.parquet.io.api.Binary;
 
 public class BinaryPlainValuesReader extends ValuesReader {
   private static final Log LOG = Log.getLog(BinaryPlainValuesReader.class);
-  private byte[] in;
+  private ByteBuffer in;
   private int offset;
 
   @Override
@@ -39,7 +40,7 @@ public class BinaryPlainValuesReader extends ValuesReader {
       int length = BytesUtils.readIntLittleEndian(in, offset);
       int start = offset + 4;
       offset = start + length;
-      return Binary.fromConstantByteArray(in, start, length);
+      return Binary.fromConstantByteBuffer(in, start, length);
     } catch (IOException e) {
       throw new ParquetDecodingException("could not read bytes at offset " + offset, e);
     } catch (RuntimeException e) {
@@ -60,11 +61,10 @@ public class BinaryPlainValuesReader extends ValuesReader {
   }
 
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset)
+  public void initFromPage(int valueCount, ByteBuffer in, int offset)
       throws IOException {
-    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.length - offset));
+    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset));
     this.in = in;
     this.offset = offset;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java
index 31e711f..c330490 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesReader.java
@@ -22,6 +22,7 @@ import static org.apache.parquet.Log.DEBUG;
 import static org.apache.parquet.column.values.bitpacking.Packer.LITTLE_ENDIAN;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.Log;
 import org.apache.parquet.column.values.ValuesReader;
@@ -62,8 +63,8 @@ public class BooleanPlainValuesReader extends ValuesReader {
    * @see org.apache.parquet.column.values.ValuesReader#initFromPage(byte[], int)
    */
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset) throws IOException {
-    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.length - offset));
+  public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException {
+    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset));
     this.in.initFromPage(valueCount, in, offset);
   }
   

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesWriter.java
index 78920f0..c3e88ea 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BooleanPlainValuesWriter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.parquet.column.values.plain;
 
+
 import static org.apache.parquet.column.Encoding.PLAIN;
 import static org.apache.parquet.column.values.bitpacking.Packer.LITTLE_ENDIAN;
 import org.apache.parquet.bytes.BytesInput;
@@ -61,6 +62,11 @@ public class BooleanPlainValuesWriter extends ValuesWriter {
   }
 
   @Override
+  public void close() {
+    bitPackingWriter.close();
+  }
+
+  @Override
   public long getAllocatedSize() {
     return bitPackingWriter.getAllocatedSize();
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
index 098a486..8496e7e 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.column.values.plain;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import org.apache.parquet.Log;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.io.ParquetDecodingException;
@@ -33,7 +34,7 @@ import static org.apache.parquet.Log.DEBUG;
  */
 public class FixedLenByteArrayPlainValuesReader extends ValuesReader {
   private static final Log LOG = Log.getLog(FixedLenByteArrayPlainValuesReader.class);
-  private byte[] in;
+  private ByteBuffer in;
   private int offset;
   private int length;
 
@@ -46,7 +47,7 @@ public class FixedLenByteArrayPlainValuesReader extends ValuesReader {
     try {
       int start = offset;
       offset = start + length;
-      return Binary.fromConstantByteArray(in, start, length);
+      return Binary.fromConstantByteBuffer(in, start, length);
     } catch (RuntimeException e) {
       throw new ParquetDecodingException("could not read bytes at offset " + offset, e);
     }
@@ -58,9 +59,9 @@ public class FixedLenByteArrayPlainValuesReader extends ValuesReader {
   }
 
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset)
+  public void initFromPage(int valueCount, ByteBuffer in, int offset)
       throws IOException {
-    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.length - offset));
+    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset));
     this.in = in;
     this.offset = offset;
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java
index 986ae0b..6ab2dea 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.values.plain;
 
 import java.io.IOException;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
@@ -40,10 +41,13 @@ public class FixedLenByteArrayPlainValuesWriter extends ValuesWriter {
   private CapacityByteArrayOutputStream arrayOut;
   private LittleEndianDataOutputStream out;
   private int length;
+  private ByteBufferAllocator allocator;
+  
 
-  public FixedLenByteArrayPlainValuesWriter(int length, int initialSize, int pageSize) {
+  public FixedLenByteArrayPlainValuesWriter(int length, int initialSize, int pageSize, ByteBufferAllocator allocator) {
     this.length = length;
-    this.arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize);
+    this.allocator = allocator;
+    this.arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize, this.allocator);
     this.out = new LittleEndianDataOutputStream(arrayOut);
   }
 
@@ -82,6 +86,11 @@ public class FixedLenByteArrayPlainValuesWriter extends ValuesWriter {
   }
 
   @Override
+  public void close() {
+    arrayOut.close();
+  }
+
+  @Override
   public long getAllocatedSize() {
     return arrayOut.getCapacity();
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
index bd938ee..c8fb303 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
@@ -20,9 +20,10 @@ package org.apache.parquet.column.values.plain;
 
 import static org.apache.parquet.Log.DEBUG;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.LittleEndianDataInputStream;
 import org.apache.parquet.column.values.ValuesReader;
@@ -41,12 +42,16 @@ abstract public class PlainValuesReader extends ValuesReader {
 
   /**
    * {@inheritDoc}
-   * @see org.apache.parquet.column.values.ValuesReader#initFromPage(byte[], int)
+   * @see org.apache.parquet.column.values.ValuesReader#initFromPage(int, ByteBuffer, int)
    */
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset) throws IOException {
-    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.length - offset));
-    this.in = new LittleEndianDataInputStream(new ByteArrayInputStream(in, offset, in.length - offset));
+  public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException {
+    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset));
+    this.in = new LittleEndianDataInputStream(toInputStream(in, offset));
+  }
+
+  private ByteBufferInputStream toInputStream(ByteBuffer in, int offset) {
+    return new ByteBufferInputStream(in.duplicate(), offset, in.limit() - offset);
   }
 
   public static class DoublePlainValuesReader extends PlainValuesReader {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
index f33bd81..add5495 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
@@ -21,6 +21,7 @@ package org.apache.parquet.column.values.plain;
 import java.io.IOException;
 import java.nio.charset.Charset;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
@@ -44,8 +45,8 @@ public class PlainValuesWriter extends ValuesWriter {
   private CapacityByteArrayOutputStream arrayOut;
   private LittleEndianDataOutputStream out;
 
-  public PlainValuesWriter(int initialSize, int pageSize) {
-    arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize);
+  public PlainValuesWriter(int initialSize, int pageSize, ByteBufferAllocator allocator) {
+    arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize, allocator);
     out = new LittleEndianDataOutputStream(arrayOut);
   }
 
@@ -126,6 +127,12 @@ public class PlainValuesWriter extends ValuesWriter {
   }
 
   @Override
+  public void close() {
+    arrayOut.close();
+    out.close();
+  }
+
+  @Override
   public long getAllocatedSize() {
     return arrayOut.getCapacity();
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
index 38eb354..1280e8d 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridDecoder.java
@@ -20,10 +20,12 @@ package org.apache.parquet.column.values.rle;
 
 import static org.apache.parquet.Log.DEBUG;
 
-import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.Log;
 import org.apache.parquet.Preconditions;
 import org.apache.parquet.bytes.BytesUtils;
@@ -43,14 +45,14 @@ public class RunLengthBitPackingHybridDecoder {
 
   private final int bitWidth;
   private final BytePacker packer;
-  private final ByteArrayInputStream in;
+  private final InputStream in;
 
   private MODE mode;
   private int currentCount;
   private int currentValue;
   private int[] currentBuffer;
 
-  public RunLengthBitPackingHybridDecoder(int bitWidth, ByteArrayInputStream in) {
+  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");

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java
index 9d37574..001d3f6 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridEncoder.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.values.rle;
 
 import java.io.IOException;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.Preconditions;
 import org.apache.parquet.bytes.BytesInput;
@@ -116,7 +117,7 @@ public class RunLengthBitPackingHybridEncoder {
 
   private boolean toBytesCalled;
 
-  public RunLengthBitPackingHybridEncoder(int bitWidth, int initialCapacity, int pageSize) {
+  public RunLengthBitPackingHybridEncoder(int bitWidth, int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
     if (DEBUG) {
       LOG.debug(String.format("Encoding: RunLengthBitPackingHybridEncoder with "
         + "bithWidth: %d initialCapacity %d", bitWidth, initialCapacity));
@@ -125,7 +126,7 @@ public class RunLengthBitPackingHybridEncoder {
     Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
 
     this.bitWidth = bitWidth;
-    this.baos = new CapacityByteArrayOutputStream(initialCapacity, pageSize);
+    this.baos = new CapacityByteArrayOutputStream(initialCapacity, pageSize, allocator);
     this.packBuffer = new byte[bitWidth];
     this.bufferedValues = new int[8];
     this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
@@ -281,6 +282,11 @@ public class RunLengthBitPackingHybridEncoder {
     reset(true);
   }
 
+  public void close() {
+    reset(false);
+    baos.close();
+  }
+
   public long getBufferedSize() {
     return baos.size();
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
index bd4e11d..4ccf2b8 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesReader.java
@@ -18,9 +18,10 @@
  */
 package org.apache.parquet.column.values.rle;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.io.ParquetDecodingException;
@@ -41,8 +42,8 @@ public class RunLengthBitPackingHybridValuesReader extends ValuesReader {
   }
 
   @Override
-  public void initFromPage(int valueCountL, byte[] page, int offset) throws IOException {
-    ByteArrayInputStream in = new ByteArrayInputStream(page, offset, page.length - offset);
+  public void initFromPage(int valueCountL, ByteBuffer page, int offset) throws IOException {
+    ByteBufferInputStream in = new ByteBufferInputStream(page, offset, page.limit() - offset);
     int length = BytesUtils.readIntLittleEndian(in);
 
     decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
index bccfd34..14ef161 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.values.rle;
 
 import java.io.IOException;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Ints;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.Encoding;
@@ -32,8 +33,8 @@ import org.apache.parquet.io.ParquetEncodingException;
 public class RunLengthBitPackingHybridValuesWriter extends ValuesWriter {
   private final RunLengthBitPackingHybridEncoder encoder;
 
-  public RunLengthBitPackingHybridValuesWriter(int bitWidth, int initialCapacity, int pageSize) {
-    this.encoder = new RunLengthBitPackingHybridEncoder(bitWidth, initialCapacity, pageSize);
+  public RunLengthBitPackingHybridValuesWriter(int bitWidth, int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
+    this.encoder = new RunLengthBitPackingHybridEncoder(bitWidth, initialCapacity, pageSize, allocator);
   }
 
   @Override
@@ -82,6 +83,11 @@ public class RunLengthBitPackingHybridValuesWriter extends ValuesWriter {
   }
 
   @Override
+  public void close() {
+    encoder.close();
+  }
+
+  @Override
   public String memUsageString(String prefix) {
     return String.format("%s RunLengthBitPackingHybrid %d bytes", prefix, getAllocatedSize());
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/io/api/Binary.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/io/api/Binary.java b/parquet-column/src/main/java/org/apache/parquet/io/api/Binary.java
index f88d740..ff833ec 100644
--- a/parquet-column/src/main/java/org/apache/parquet/io/api/Binary.java
+++ b/parquet-column/src/main/java/org/apache/parquet/io/api/Binary.java
@@ -62,12 +62,16 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
 
   abstract boolean equals(byte[] bytes, int offset, int length);
 
+  abstract boolean equals(ByteBuffer bytes, int offset, int length);
+
   abstract boolean equals(Binary other);
 
   abstract public int compareTo(Binary other);
 
   abstract int compareTo(byte[] bytes, int offset, int length);
 
+  abstract int compareTo(ByteBuffer bytes, int offset, int length);
+
   abstract public ByteBuffer toByteBuffer();
 
   @Override
@@ -174,6 +178,11 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
     }
 
     @Override
+    boolean equals(ByteBuffer bytes, int otherOffset, int otherLength) {
+      return Binary.equals(value, offset, length, bytes, otherOffset, otherLength);
+    }
+
+    @Override
     public int compareTo(Binary other) {
       return other.compareTo(value, offset, length);
     }
@@ -184,6 +193,11 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
     }
 
     @Override
+    int compareTo(ByteBuffer bytes, int otherOffset, int otherLength) {
+      return Binary.compareByteArrayToByteBuffer(value, offset, length, bytes, otherOffset, otherLength);
+    }
+
+    @Override
     public ByteBuffer toByteBuffer() {
       return ByteBuffer.wrap(value, offset, length);
     }
@@ -292,6 +306,11 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
     }
 
     @Override
+    boolean equals(ByteBuffer bytes, int otherOffset, int otherLength) {
+      return Binary.equals(value, 0, value.length, bytes, otherOffset, otherLength);
+    }
+
+    @Override
     public int compareTo(Binary other) {
       return other.compareTo(value, 0, value.length);
     }
@@ -302,6 +321,11 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
     }
 
     @Override
+    int compareTo(ByteBuffer bytes, int otherOffset, int otherLength) {
+      return Binary.compareByteArrayToByteBuffer(value, 0, value.length, bytes, otherOffset, otherLength);
+    }
+
+    @Override
     public ByteBuffer toByteBuffer() {
       return ByteBuffer.wrap(value);
     }
@@ -330,36 +354,58 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
   }
 
   private static class ByteBufferBackedBinary extends Binary {
-    private transient ByteBuffer value;
-    private transient byte[] cachedBytes;
+    private ByteBuffer value;
+    private byte[] cachedBytes;
+    private final int offset;
+    private final int length;
 
-    public ByteBufferBackedBinary(ByteBuffer value, boolean isBackingBytesReused) {
+    public ByteBufferBackedBinary(ByteBuffer value, int offset, int length, boolean isBackingBytesReused) {
       this.value = value;
+      this.offset = offset;
+      this.length = length;
       this.isBackingBytesReused = isBackingBytesReused;
     }
 
     @Override
     public String toStringUsingUTF8() {
-      return UTF8.decode(value).toString();
+      int limit = value.limit();
+      value.limit(offset+length);
+      int position = value.position();
+      value.position(offset);
+      // no corresponding interface to read a subset of a buffer, would have to slice it
+      // which creates another ByteBuffer object or do what is done here to adjust the
+      // limit/offset and set them back after
+      String ret = UTF8.decode(value).toString();
+      value.limit(limit);
+      value.position(position);
+      return ret;
     }
 
     @Override
     public int length() {
-      return value.remaining();
+      return length;
     }
 
     @Override
     public void writeTo(OutputStream out) throws IOException {
-      // TODO: should not have to materialize those bytes
-      out.write(getBytesUnsafe());
+      if (value.hasArray()) {
+        out.write(value.array(), value.arrayOffset() + offset, length);
+      } else {
+        out.write(getBytesUnsafe(), 0, length);
+      }
     }
 
     @Override
     public byte[] getBytes() {
-      byte[] bytes = new byte[value.remaining()];
+      byte[] bytes = new byte[length];
 
-      value.mark();
-      value.get(bytes).reset();
+      int limit = value.limit();
+      value.limit(offset + length);
+      int position = value.position();
+      value.position(offset);
+      value.get(bytes);
+      value.limit(limit);
+      value.position(position);
       if (!isBackingBytesReused) { // backing buffer might change
         cachedBytes = bytes;
       }
@@ -375,60 +421,68 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
     public Binary slice(int start, int length) {
       return Binary.fromConstantByteArray(getBytesUnsafe(), start, length);
     }
-
     @Override
     public int hashCode() {
       if (value.hasArray()) {
-        return Binary.hashCode(value.array(), value.arrayOffset() + value.position(),
-            value.arrayOffset() + value.remaining());
+        return Binary.hashCode(value.array(), value.arrayOffset() + offset, length);
+      } else {
+        return Binary.hashCode(value, offset, length);
       }
-      byte[] bytes = getBytesUnsafe();
-      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());
+        return other.equals(value.array(), value.arrayOffset() + offset, length);
+      } else {
+        return other.equals(value, offset, length);
       }
-      byte[] bytes = getBytesUnsafe();
-      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);
+        return Binary.equals(value.array(), value.arrayOffset() + offset, length, other, otherOffset, otherLength);
+      } else {
+        return Binary.equals(other, otherOffset, otherLength, value, offset, length);
       }
-      byte[] bytes = getBytesUnsafe();
-      return Binary.equals(bytes, 0, bytes.length, other, otherOffset, otherLength);
+    }
+
+    @Override
+    boolean equals(ByteBuffer otherBytes, int otherOffset, int otherLength) {
+      return Binary.equals(value, 0, length, otherBytes, otherOffset, otherLength);
     }
 
     @Override
     public int compareTo(Binary other) {
       if (value.hasArray()) {
-        return other.compareTo(value.array(), value.arrayOffset() + value.position(),
-            value.arrayOffset() + value.remaining());
+        return other.compareTo(value.array(), value.arrayOffset() + offset, length);
+      } else {
+        return other.compareTo(value, offset, length);
       }
-      byte[] bytes = getBytesUnsafe();
-      return other.compareTo(bytes, 0, bytes.length);
     }
 
     @Override
     int compareTo(byte[] other, int otherOffset, int otherLength) {
       if (value.hasArray()) {
-        return Binary.compareTwoByteArrays(value.array(), value.arrayOffset() + value.position(),
-            value.arrayOffset() + value.remaining(), other, otherOffset, otherLength);
+        return Binary.compareTwoByteArrays(value.array(), value.arrayOffset() + offset, length,
+            other, otherOffset, otherLength);
+      } {
+        return Binary.compareByteBufferToByteArray(value, offset, length, other, otherOffset, otherLength);
       }
-      byte[] bytes = getBytesUnsafe();
-      return Binary.compareTwoByteArrays(bytes, 0, bytes.length, other, otherOffset, otherLength);
+    }
+
+    @Override
+    int compareTo(ByteBuffer bytes, int otherOffset, int otherLength) {
+      return Binary.compareTwoByteBuffers(value, offset, length, bytes, otherOffset, otherLength);
     }
 
     @Override
     public ByteBuffer toByteBuffer() {
-      return value;
+      ByteBuffer ret = value.slice();
+      ret.position(offset);
+      ret.limit(offset + length);
+      return ret;
     }
 
     @Override
@@ -456,12 +510,20 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
 
   }
 
+  public static Binary fromReusedByteBuffer(final ByteBuffer value, int offset, int length) {
+    return new ByteBufferBackedBinary(value, offset, length, true);
+  }
+
+  public static Binary fromConstantByteBuffer(final ByteBuffer value, int offset, int length) {
+    return new ByteBufferBackedBinary(value, offset, length, false);
+  }
+
   public static Binary fromReusedByteBuffer(final ByteBuffer value) {
-    return new ByteBufferBackedBinary(value, true);
+    return new ByteBufferBackedBinary(value, value.position(), value.remaining(), true);
   }
 
   public static Binary fromConstantByteBuffer(final ByteBuffer value) {
-    return new ByteBufferBackedBinary(value, false);
+    return new ByteBufferBackedBinary(value, value.position(), value.remaining(), false);
   }
 
   @Deprecated
@@ -492,6 +554,39 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
     return result;
   }
 
+  private static final int hashCode(ByteBuffer buf, int offset, int length) {
+    int result = 1;
+    for (int i = offset; i < offset + length; i++) {
+      byte b = buf.get(i);
+      result = 31 * result + b;
+    }
+    return result;
+  }
+
+  private static final boolean equals(ByteBuffer buf1, int offset1, int length1, ByteBuffer buf2, int offset2, int length2) {
+    if (buf1 == null && buf2 == null) return true;
+    if (buf1 == null || buf2 == null) return false;
+    if (length1 != length2) return false;
+    for (int i = 0; i < length1; i++) {
+      if (buf1.get(i + offset1) != buf2.get(i + offset2)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static final boolean equals(byte[] array1, int offset1, int length1, ByteBuffer buf, int offset2, int length2) {
+    if (array1 == null && buf == null) return true;
+    if (array1 == null || buf == null) return false;
+    if (length1 != length2) return false;
+    for (int i = 0; i < length1; i++) {
+      if (array1[i + offset1] != buf.get(i + offset2)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   /**
    * @see {@link Arrays#equals(byte[], byte[])}
    * @param array1
@@ -515,6 +610,47 @@ abstract public class Binary implements Comparable<Binary>, Serializable {
     return true;
   }
 
+  private static final int compareByteBufferToByteArray(ByteBuffer buf, int offset1, int length1,
+                                                        byte[] array, int offset2, int length2) {
+    return -1 * Binary.compareByteArrayToByteBuffer(array, offset1, length1, buf, offset2, length2);
+  }
+
+  private static final int compareByteArrayToByteBuffer(byte[] array1, int offset1, int length1,
+                                                        ByteBuffer buf, int offset2, int length2) {
+    if (array1 == null && buf == null) return 0;
+    int min_length = (length1 < length2) ? length1 : length2;
+    for (int i = 0; i < min_length; i++) {
+      if (array1[i + offset1] < buf.get(i + offset2)) {
+        return 1;
+      }
+      if (array1[i + offset1] > buf.get(i + offset2)) {
+        return -1;
+      }
+    }
+    // check remainder
+    if (length1 == length2) { return 0; }
+    else if (length1 < length2) { return 1;}
+    else { return -1; }
+  }
+
+  private static final int compareTwoByteBuffers(ByteBuffer buf1, int offset1, int length1,
+                                                        ByteBuffer buf2, int offset2, int length2) {
+    if (buf1 == null && buf2 == null) return 0;
+    int min_length = (length1 < length2) ? length1 : length2;
+    for (int i = 0; i < min_length; i++) {
+      if (buf1.get(i + offset1) < buf2.get(i + offset2)) {
+        return 1;
+      }
+      if (buf1.get(i + offset1) > buf2.get(i + offset2)) {
+        return -1;
+      }
+    }
+    // check remainder
+    if (length1 == length2) { return 0; }
+    else if (length1 < length2) { return 1;}
+    else { return -1; }
+  }
+
   private static final int compareTwoByteArrays(byte[] array1, int offset1, int length1,
                                                 byte[] array2, int offset2, int length2) {
     if (array1 == null && array2 == null) return 0;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java b/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java
index 7988f4a..5c6e460 100644
--- a/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java
@@ -333,10 +333,9 @@ public final class PrimitiveType extends Type {
    * @param decimalMeta (optional) metadata about the decimal type
    * @param id the id of the field
    */
-  PrimitiveType(
-      Repetition repetition, PrimitiveTypeName primitive,
-      int length, String name, OriginalType originalType,
-      DecimalMetadata decimalMeta, ID id) {
+  public PrimitiveType(Repetition repetition, PrimitiveTypeName primitive,
+                       int length, String name, OriginalType originalType,
+                       DecimalMetadata decimalMeta, ID id) {
     super(name, repetition, originalType, id);
     this.primitive = primitive;
     this.length = length;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java b/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java
index a1820e6..6792361 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import org.apache.parquet.Version;
 import org.apache.parquet.VersionParser;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import org.junit.Test;
 
 import org.apache.parquet.column.ColumnDescriptor;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/impl/TestCorruptDeltaByteArrays.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/impl/TestCorruptDeltaByteArrays.java b/parquet-column/src/test/java/org/apache/parquet/column/impl/TestCorruptDeltaByteArrays.java
index 0327948..9bb2759 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/impl/TestCorruptDeltaByteArrays.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/impl/TestCorruptDeltaByteArrays.java
@@ -36,8 +36,10 @@ import org.apache.parquet.io.api.PrimitiveConverter;
 import org.apache.parquet.schema.PrimitiveType;
 import org.junit.Assert;
 import org.junit.Test;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 
 import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -74,16 +76,20 @@ public class TestCorruptDeltaByteArrays {
     assertFalse(CorruptDeltaByteArrays.requiresSequentialReads(fixed, Encoding.DELTA_BYTE_ARRAY));
   }
 
+  private DeltaByteArrayWriter getDeltaByteArrayWriter() {
+    return new DeltaByteArrayWriter(10, 100, new HeapByteBufferAllocator());
+  }
+
   @Test
   public void testReassemblyWithCorruptPage() throws Exception {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(10, 100);
+    DeltaByteArrayWriter writer = getDeltaByteArrayWriter();
 
     String lastValue = null;
     for (int i = 0; i < 10; i += 1) {
       lastValue = str(i);
       writer.writeBytes(Binary.fromString(lastValue));
     }
-    byte[] firstPageBytes = writer.getBytes().toByteArray();
+    ByteBuffer firstPageBytes = writer.getBytes().toByteBuffer();
 
     writer.reset(); // sets previous to new byte[0]
     corruptWriter(writer, lastValue);
@@ -91,7 +97,7 @@ public class TestCorruptDeltaByteArrays {
     for (int i = 10; i < 20; i += 1) {
       writer.writeBytes(Binary.fromString(str(i)));
     }
-    byte[] corruptPageBytes = writer.getBytes().toByteArray();
+    ByteBuffer corruptPageBytes = writer.getBytes().toByteBuffer();
 
     DeltaByteArrayReader firstPageReader = new DeltaByteArrayReader();
     firstPageReader.initFromPage(10, firstPageBytes, 0);
@@ -119,19 +125,19 @@ public class TestCorruptDeltaByteArrays {
 
   @Test
   public void testReassemblyWithoutCorruption() throws Exception {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(10, 100);
+    DeltaByteArrayWriter writer = getDeltaByteArrayWriter();
 
     for (int i = 0; i < 10; i += 1) {
       writer.writeBytes(Binary.fromString(str(i)));
     }
-    byte[] firstPageBytes = writer.getBytes().toByteArray();
+    ByteBuffer firstPageBytes = writer.getBytes().toByteBuffer();
 
     writer.reset(); // sets previous to new byte[0]
 
     for (int i = 10; i < 20; i += 1) {
       writer.writeBytes(Binary.fromString(str(i)));
     }
-    byte[] secondPageBytes = writer.getBytes().toByteArray();
+    ByteBuffer secondPageBytes = writer.getBytes().toByteBuffer();
 
     DeltaByteArrayReader firstPageReader = new DeltaByteArrayReader();
     firstPageReader.initFromPage(10, firstPageBytes, 0);
@@ -150,19 +156,19 @@ public class TestCorruptDeltaByteArrays {
 
   @Test
   public void testOldReassemblyWithoutCorruption() throws Exception {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(10, 100);
+    DeltaByteArrayWriter writer = getDeltaByteArrayWriter();
 
     for (int i = 0; i < 10; i += 1) {
       writer.writeBytes(Binary.fromString(str(i)));
     }
-    byte[] firstPageBytes = writer.getBytes().toByteArray();
+    ByteBuffer firstPageBytes = writer.getBytes().toByteBuffer();
 
     writer.reset(); // sets previous to new byte[0]
 
     for (int i = 10; i < 20; i += 1) {
       writer.writeBytes(Binary.fromString(str(i)));
     }
-    byte[] secondPageBytes = writer.getBytes().toByteArray();
+    ByteBuffer secondPageBytes = writer.getBytes().toByteBuffer();
 
     DeltaByteArrayReader firstPageReader = new DeltaByteArrayReader();
     firstPageReader.initFromPage(10, firstPageBytes, 0);
@@ -185,15 +191,16 @@ public class TestCorruptDeltaByteArrays {
     MemPageStore pages = new MemPageStore(0);
     PageWriter memWriter = pages.getPageWriter(column);
 
+    ParquetProperties parquetProps = new ParquetProperties(0, ParquetProperties.WriterVersion.PARQUET_1_0, false, new HeapByteBufferAllocator());
+
     // get generic repetition and definition level bytes to use for pages
-    ValuesWriter rdValues = ParquetProperties
-        .getColumnDescriptorValuesWriter(0, 10, 100);
+    ValuesWriter rdValues = parquetProps.getColumnDescriptorValuesWriter(0, 10, 100);
     for (int i = 0; i < 10; i += 1) {
       rdValues.writeInteger(0);
     }
     // use a byte array backed BytesInput because it is reused
     BytesInput rd = BytesInput.from(rdValues.getBytes().toByteArray());
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(10, 100);
+    DeltaByteArrayWriter writer = getDeltaByteArrayWriter();
     String lastValue = null;
     List<String> values = new ArrayList<String>();
     for (int i = 0; i < 10; i += 1) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java b/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
index 135123f..044fe2a 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.mem;
 
 import static org.junit.Assert.assertEquals;
 
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import org.junit.Test;
 
 import org.apache.parquet.Log;
@@ -160,6 +161,6 @@ public class TestMemColumn {
   }
 
   private ColumnWriteStoreV1 newColumnWriteStoreImpl(MemPageStore memPageStore) {
-    return new ColumnWriteStoreV1(memPageStore, 2048, 2048, false, WriterVersion.PARQUET_1_0);
+    return new ColumnWriteStoreV1(memPageStore, 2048, 2048, false, WriterVersion.PARQUET_1_0, new HeapByteBufferAllocator());
   }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
index d5bfe22..ddab636 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
@@ -109,5 +109,4 @@ public class MemPageWriter implements PageWriter {
     return String.format("%s %,d bytes", prefix, memSize);
 
   }
-
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/Utils.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/Utils.java b/parquet-column/src/test/java/org/apache/parquet/column/values/Utils.java
index c9a62b4..8caad2b 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/Utils.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/Utils.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.column.values;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Random;
 
 import org.apache.parquet.io.api.Binary;
@@ -61,7 +62,7 @@ public class Utils {
   public static Binary[] readData(ValuesReader reader, byte[] data, int offset, int length)
       throws IOException {
     Binary[] bins = new Binary[length];
-    reader.initFromPage(length, data, 0);
+    reader.initFromPage(length, ByteBuffer.wrap(data), 0);
     for(int i=0; i < length; i++) {
       bins[i] = reader.readBytes();
     }
@@ -76,7 +77,7 @@ public class Utils {
   public static int[] readInts(ValuesReader reader, byte[] data, int offset, int length)
       throws IOException {
     int[] ints = new int[length];
-    reader.initFromPage(length, data, offset);
+    reader.initFromPage(length, ByteBuffer.wrap(data), offset);
     for(int i=0; i < length; i++) {
       ints[i] = reader.readInteger();
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/BitPackingPerfTest.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/BitPackingPerfTest.java b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/BitPackingPerfTest.java
index e74e787..2733b72 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/BitPackingPerfTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/BitPackingPerfTest.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.values.bitpacking;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.bitpacking.BitPacking.BitPackingWriter;
@@ -87,7 +88,7 @@ public class BitPackingPerfTest {
     System.out.print(" no gc <");
     for (int k = 0; k < N; k++) {
       long t2 = System.nanoTime();
-      r.initFromPage(result.length, bytes, 0);
+      r.initFromPage(result.length, ByteBuffer.wrap(bytes), 0);
       for (int i = 0; i < result.length; i++) {
         result[i] = r.readInteger();
       }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
index 2f311ec..aef259c 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
@@ -23,9 +23,11 @@ import static org.junit.Assert.assertEquals;
 import static org.apache.parquet.column.values.bitpacking.Packer.BIG_ENDIAN;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.junit.Test;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.ValuesWriter;
@@ -172,7 +174,7 @@ public class TestBitPackingColumn {
       LOG.debug("bytes: " + TestBitPacking.toString(bytes));
       assertEquals(type.toString(), expected, TestBitPacking.toString(bytes));
       ValuesReader r = type.getReader(bound);
-      r.initFromPage(vals.length, bytes, 0);
+      r.initFromPage(vals.length, ByteBuffer.wrap(bytes), 0);
       int[] result = new int[vals.length];
       for (int i = 0; i < result.length; i++) {
         result[i] = r.readInteger();
@@ -188,7 +190,7 @@ public class TestBitPackingColumn {
         return new BitPackingValuesReader(bound);
       }
       public ValuesWriter getWriter(final int bound) {
-        return new BitPackingValuesWriter(bound, 32*1024, 64*1024);
+        return new BitPackingValuesWriter(bound, 32*1024, 64*1024, new DirectByteBufferAllocator());
       }
     }
     ,

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/boundedint/TestBoundedColumns.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/boundedint/TestBoundedColumns.java b/parquet-column/src/test/java/org/apache/parquet/column/values/boundedint/TestBoundedColumns.java
index ba979b7..d1e43d2 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/boundedint/TestBoundedColumns.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/boundedint/TestBoundedColumns.java
@@ -23,11 +23,14 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Random;
 
 import org.junit.Test;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
+
 public class TestBoundedColumns {
   private final Random r = new Random(42L);
 
@@ -54,7 +57,7 @@ public class TestBoundedColumns {
   }
 
   private void compareOutput(int bound, int[] ints, String[] result) throws IOException {
-    BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound, 64*1024, 64*1024);
+    BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound, 64*1024, 64*1024, new DirectByteBufferAllocator());
     for (int i : ints) {
       bicw.writeInteger(i);
     }
@@ -63,7 +66,7 @@ public class TestBoundedColumns {
     byte[] byteArray = bicw.getBytes().toByteArray();
     assertEquals(concat(result), toBinaryString(byteArray, 4));
     BoundedIntValuesReader bicr = new BoundedIntValuesReader(bound);
-    bicr.initFromPage(1, byteArray, 0);
+    bicr.initFromPage(1, ByteBuffer.wrap(byteArray), 0);
     String expected = "";
     String got = "";
     for (int i : ints) {
@@ -123,7 +126,7 @@ public class TestBoundedColumns {
       ByteArrayOutputStream tmp = new ByteArrayOutputStream();
 
       int[] stream = new int[totalValuesInStream];
-      BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound, 64 * 1024, 64*1024);
+      BoundedIntValuesWriter bicw = new BoundedIntValuesWriter(bound, 64 * 1024, 64*1024, new DirectByteBufferAllocator());
       int idx = 0;
       for (int stripeNum = 0; stripeNum < valuesPerStripe.length; stripeNum++) {
         int next = 0;
@@ -155,7 +158,7 @@ public class TestBoundedColumns {
       idx = 0;
       int offset = 0;
       for (int stripeNum = 0; stripeNum < valuesPerStripe.length; stripeNum++) {
-        bicr.initFromPage(1, input, offset);
+        bicr.initFromPage(1, ByteBuffer.wrap(input), offset);
         offset = bicr.getNextOffset();
         for (int i = 0; i < valuesPerStripe[stripeNum]; i++) {
           int number = stream[idx++];

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java
index d428fbf..6308e47 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterTest.java
@@ -22,11 +22,13 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Random;
 
 import org.junit.Before;
 import org.junit.Test;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.values.ValuesWriter;
 import org.apache.parquet.io.ParquetDecodingException;
@@ -42,13 +44,13 @@ public class DeltaBinaryPackingValuesWriterTest {
   public void setUp() {
     blockSize = 128;
     miniBlockNum = 4;
-    writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 200);
+    writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 200, new DirectByteBufferAllocator());
     random = new Random();
   }
 
   @Test(expected = IllegalArgumentException.class)
   public void miniBlockSizeShouldBeMultipleOf8() {
-    new DeltaBinaryPackingValuesWriter(1281, 4, 100, 100);
+    new DeltaBinaryPackingValuesWriter(1281, 4, 100, 100, new DirectByteBufferAllocator());
   }
 
   /* When data size is multiple of Block*/
@@ -154,7 +156,7 @@ public class DeltaBinaryPackingValuesWriterTest {
     System.arraycopy(valueContent, 0, pageContent, contentOffsetInPage, valueContent.length);
 
     //offset should be correct
-    reader.initFromPage(100, pageContent, contentOffsetInPage);
+    reader.initFromPage(100, ByteBuffer.wrap(pageContent), contentOffsetInPage);
     int offset= reader.getNextOffset();
     assertEquals(valueContent.length + contentOffsetInPage, offset);
 
@@ -187,7 +189,7 @@ public class DeltaBinaryPackingValuesWriterTest {
     }
     writeData(data);
     reader = new DeltaBinaryPackingValuesReader();
-    reader.initFromPage(100, writer.getBytes().toByteArray(), 0);
+    reader.initFromPage(100, writer.getBytes().toByteBuffer(), 0);
     for (int i = 0; i < data.length; i++) {
       if (i % 3 == 0) {
         reader.skip();
@@ -243,7 +245,7 @@ public class DeltaBinaryPackingValuesWriterTest {
         + blockFlushed * miniBlockNum //bitWidth of mini blocks
         + (5.0 * blockFlushed);//min delta for each block
     assertTrue(estimatedSize >= page.length);
-    reader.initFromPage(100, page, 0);
+    reader.initFromPage(100, ByteBuffer.wrap(page), 0);
 
     for (int i = 0; i < length; i++) {
       assertEquals(data[i], reader.readInteger());

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java
index dc69fcc..40f6bfc 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkIntegerOutputSize.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.column.values.delta.benchmark;
 
 import org.junit.Test;
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriter;
 import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
 import java.util.Random;
@@ -77,8 +78,8 @@ public class BenchmarkIntegerOutputSize {
   }
 
   public void testRandomIntegers(IntFunc func,int bitWidth) {
-    DeltaBinaryPackingValuesWriter delta=new DeltaBinaryPackingValuesWriter(blockSize,miniBlockNum, 100, 20000);
-    RunLengthBitPackingHybridValuesWriter rle= new RunLengthBitPackingHybridValuesWriter(bitWidth, 100, 20000);
+    DeltaBinaryPackingValuesWriter delta=new DeltaBinaryPackingValuesWriter(blockSize,miniBlockNum, 100, 20000, new DirectByteBufferAllocator());
+    RunLengthBitPackingHybridValuesWriter rle= new RunLengthBitPackingHybridValuesWriter(bitWidth, 100, 20000, new DirectByteBufferAllocator());
     for (int i = 0; i < dataSize; i++) {
       int v = func.getIntValue();
       delta.writeInteger(v);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java
index 24b007f..4ad5dad 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/BenchmarkReadingRandomIntegers.java
@@ -25,6 +25,7 @@ import com.carrotsearch.junitbenchmarks.annotation.BenchmarkMethodChart;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.ValuesWriter;
 import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader;
@@ -33,6 +34,7 @@ import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridValuesReade
 import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Random;
 
 @AxisRange(min = 0, max = 1)
@@ -54,8 +56,8 @@ public class BenchmarkReadingRandomIntegers {
       data[i] = random.nextInt(100) - 200;
     }
 
-    ValuesWriter delta = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 20000);
-    ValuesWriter rle = new RunLengthBitPackingHybridValuesWriter(32, 100, 20000);
+    ValuesWriter delta = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 20000, new DirectByteBufferAllocator());
+    ValuesWriter rle = new RunLengthBitPackingHybridValuesWriter(32, 100, 20000, new DirectByteBufferAllocator());
 
     for (int i = 0; i < data.length; i++) {
       delta.writeInteger(data[i]);
@@ -86,7 +88,7 @@ public class BenchmarkReadingRandomIntegers {
   }
 
   private void readData(ValuesReader reader, byte[] deltaBytes) throws IOException {
-    reader.initFromPage(data.length, deltaBytes, 0);
+    reader.initFromPage(data.length, ByteBuffer.wrap(deltaBytes), 0);
     for (int i = 0; i < data.length; i++) {
       reader.readInteger();
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java
index 50c97cf..80e6533 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/RandomWritingBenchmarkTest.java
@@ -25,6 +25,7 @@ import com.carrotsearch.junitbenchmarks.annotation.BenchmarkMethodChart;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.ValuesWriter;
 import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriter;
 import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
@@ -50,21 +51,21 @@ public class RandomWritingBenchmarkTest extends BenchMarkTest{
   @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
   @Test
   public void writeDeltaPackingTest(){
-    DeltaBinaryPackingValuesWriter writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 20000);
+    DeltaBinaryPackingValuesWriter writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 20000, new DirectByteBufferAllocator());
     runWriteTest(writer);
   }
 
   @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
   @Test
   public void writeRLETest(){
-    ValuesWriter writer = new RunLengthBitPackingHybridValuesWriter(32, 100, 20000);
+    ValuesWriter writer = new RunLengthBitPackingHybridValuesWriter(32, 100, 20000, new DirectByteBufferAllocator());
     runWriteTest(writer);
   }
 
   @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
   @Test
   public void writeDeltaPackingTest2(){
-    DeltaBinaryPackingValuesWriter writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 20000);
+    DeltaBinaryPackingValuesWriter writer = new DeltaBinaryPackingValuesWriter(blockSize, miniBlockNum, 100, 20000, new DirectByteBufferAllocator());
     runWriteTest(writer);
   }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java
index 3141fd7..0dc7cb0 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/benchmark/SmallRangeWritingBenchmarkTest.java
@@ -23,6 +23,7 @@ import com.carrotsearch.junitbenchmarks.annotation.AxisRange;
 import com.carrotsearch.junitbenchmarks.annotation.BenchmarkMethodChart;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.ValuesWriter;
 import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
 import java.util.Random;
@@ -42,7 +43,7 @@ public class SmallRangeWritingBenchmarkTest extends RandomWritingBenchmarkTest {
   @BenchmarkOptions(benchmarkRounds = 10, warmupRounds = 2)
   @Test
   public void writeRLEWithSmallBitWidthTest(){
-    ValuesWriter writer = new RunLengthBitPackingHybridValuesWriter(2, 100, 20000);
+    ValuesWriter writer = new RunLengthBitPackingHybridValuesWriter(2, 100, 20000, new DirectByteBufferAllocator());
     runWriteTest(writer);
   }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java b/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java
index aaae064..d7ebee5 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.junit.Test;
 import org.junit.Assert;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.Utils;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader;
@@ -32,11 +33,15 @@ public class TestDeltaLengthByteArray {
 
   String[] values = { "parquet", "hadoop", "mapreduce"};
 
+  private DeltaLengthByteArrayValuesWriter getDeltaLengthByteArrayValuesWriter() {
+    return new DeltaLengthByteArrayValuesWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
+  }
+
   @Test
   public void testSerialization () throws IOException {
-    DeltaLengthByteArrayValuesWriter writer = new DeltaLengthByteArrayValuesWriter(64 * 1024, 64 * 1024);
+    DeltaLengthByteArrayValuesWriter writer = getDeltaLengthByteArrayValuesWriter();
     DeltaLengthByteArrayValuesReader reader = new DeltaLengthByteArrayValuesReader();
-
+    
     Utils.writeData(writer, values);
     Binary[] bin = Utils.readData(reader, writer.getBytes().toByteArray(), values.length);
 
@@ -47,7 +52,7 @@ public class TestDeltaLengthByteArray {
 
   @Test
   public void testRandomStrings() throws IOException {
-    DeltaLengthByteArrayValuesWriter writer = new DeltaLengthByteArrayValuesWriter(64 * 1024, 64 * 1024);
+    DeltaLengthByteArrayValuesWriter writer = getDeltaLengthByteArrayValuesWriter();
     DeltaLengthByteArrayValuesReader reader = new DeltaLengthByteArrayValuesReader();
 
     String[] values = Utils.getRandomStringSamples(1000, 32);
@@ -61,7 +66,7 @@ public class TestDeltaLengthByteArray {
 
   @Test
   public void testLengths() throws IOException {
-    DeltaLengthByteArrayValuesWriter writer = new DeltaLengthByteArrayValuesWriter(64 * 1024, 64 * 1024);
+    DeltaLengthByteArrayValuesWriter writer = getDeltaLengthByteArrayValuesWriter();
     ValuesReader reader = new DeltaBinaryPackingValuesReader();
 
     Utils.writeData(writer, values);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/benchmark/BenchmarkDeltaLengthByteArray.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/benchmark/BenchmarkDeltaLengthByteArray.java b/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/benchmark/BenchmarkDeltaLengthByteArray.java
index f5f9d76..69c5e15 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/benchmark/BenchmarkDeltaLengthByteArray.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/benchmark/BenchmarkDeltaLengthByteArray.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.Utils;
 import org.apache.parquet.column.values.deltalengthbytearray.DeltaLengthByteArrayValuesReader;
 import org.apache.parquet.column.values.deltalengthbytearray.DeltaLengthByteArrayValuesWriter;
@@ -47,7 +48,7 @@ public class BenchmarkDeltaLengthByteArray {
   @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 4)
   @Test
   public void benchmarkRandomStringsWithPlainValuesWriter() throws IOException {
-    PlainValuesWriter writer = new PlainValuesWriter(64 * 1024, 64 * 1024);
+    PlainValuesWriter writer = new PlainValuesWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     BinaryPlainValuesReader reader = new BinaryPlainValuesReader();
 
     Utils.writeData(writer, values);
@@ -59,7 +60,7 @@ public class BenchmarkDeltaLengthByteArray {
   @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 4)
   @Test
   public void benchmarkRandomStringsWithDeltaLengthByteArrayValuesWriter() throws IOException {
-    DeltaLengthByteArrayValuesWriter writer = new DeltaLengthByteArrayValuesWriter(64 * 1024, 64 * 1024);
+    DeltaLengthByteArrayValuesWriter writer = new DeltaLengthByteArrayValuesWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     DeltaLengthByteArrayValuesReader reader = new DeltaLengthByteArrayValuesReader();
 
     Utils.writeData(writer, values);


[2/4] parquet-mr git commit: PARQUET-77: ByteBuffer use in read and write paths

Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java b/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java
index b62ef84..770f4dc 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.junit.Test;
 import org.junit.Assert;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.Utils;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader;
@@ -35,7 +36,7 @@ public class TestDeltaByteArray {
 
   @Test
   public void testSerialization () throws Exception {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024);
+    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     DeltaByteArrayReader reader = new DeltaByteArrayReader();
 
     assertReadWrite(writer, reader, values);
@@ -43,14 +44,14 @@ public class TestDeltaByteArray {
 
   @Test
   public void testRandomStrings() throws Exception {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024);
+    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     DeltaByteArrayReader reader = new DeltaByteArrayReader();
     assertReadWrite(writer, reader, randvalues);
   }
 
   @Test
   public void testLengths() throws IOException {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024);
+    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     ValuesReader reader = new DeltaBinaryPackingValuesReader();
 
     Utils.writeData(writer, values);
@@ -82,7 +83,7 @@ public class TestDeltaByteArray {
 
   @Test
   public void testWriterReset() throws Exception {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024);
+    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
 
     assertReadWrite(writer, new DeltaByteArrayReader(), values);
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/benchmark/BenchmarkDeltaByteArray.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/benchmark/BenchmarkDeltaByteArray.java b/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/benchmark/BenchmarkDeltaByteArray.java
index c61ef30..eac4bd2 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/benchmark/BenchmarkDeltaByteArray.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/benchmark/BenchmarkDeltaByteArray.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.column.values.Utils;
 import org.apache.parquet.column.values.deltastrings.DeltaByteArrayReader;
 import org.apache.parquet.column.values.deltastrings.DeltaByteArrayWriter;
@@ -54,7 +55,7 @@ public class BenchmarkDeltaByteArray {
   @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 4)
   @Test
   public void benchmarkRandomStringsWithPlainValuesWriter() throws IOException {
-    PlainValuesWriter writer = new PlainValuesWriter(64 * 1024, 64 * 1024);
+    PlainValuesWriter writer = new PlainValuesWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     BinaryPlainValuesReader reader = new BinaryPlainValuesReader();
 
     Utils.writeData(writer, values);
@@ -66,7 +67,7 @@ public class BenchmarkDeltaByteArray {
   @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 4)
   @Test
   public void benchmarkRandomStringsWithDeltaLengthByteArrayValuesWriter() throws IOException {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024);
+    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     DeltaByteArrayReader reader = new DeltaByteArrayReader();
 
     Utils.writeData(writer, values);
@@ -78,7 +79,7 @@ public class BenchmarkDeltaByteArray {
   @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 4)
   @Test
   public void benchmarkSortedStringsWithPlainValuesWriter() throws IOException {
-    PlainValuesWriter writer = new PlainValuesWriter(64 * 1024, 64 * 1024);
+    PlainValuesWriter writer = new PlainValuesWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     BinaryPlainValuesReader reader = new BinaryPlainValuesReader();
 
     Utils.writeData(writer, sortedVals);
@@ -90,7 +91,7 @@ public class BenchmarkDeltaByteArray {
   @BenchmarkOptions(benchmarkRounds = 20, warmupRounds = 4)
   @Test
   public void benchmarkSortedStringsWithDeltaLengthByteArrayValuesWriter() throws IOException {
-    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024);
+    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     DeltaByteArrayReader reader = new DeltaByteArrayReader();
 
     Utils.writeData(writer, sortedVals);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java b/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java
index 020868e..ada1c93 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java
@@ -28,10 +28,12 @@ import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.Dictionary;
@@ -54,27 +56,27 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
 public class TestDictionary {
 
   private <I extends DictionaryValuesWriter> FallbackValuesWriter<I, PlainValuesWriter> plainFallBack(I dvw, int initialSize) {
-    return FallbackValuesWriter.of(dvw, new PlainValuesWriter(initialSize, initialSize * 5));
+    return FallbackValuesWriter.of(dvw, new PlainValuesWriter(initialSize, initialSize * 5, new DirectByteBufferAllocator()));
   }
 
   private FallbackValuesWriter<PlainBinaryDictionaryValuesWriter, PlainValuesWriter> newPlainBinaryDictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
-    return plainFallBack(new PlainBinaryDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY), initialSize);
+    return plainFallBack(new PlainBinaryDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY, new DirectByteBufferAllocator()), initialSize);
   }
 
   private FallbackValuesWriter<PlainLongDictionaryValuesWriter, PlainValuesWriter> newPlainLongDictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
-    return plainFallBack(new PlainLongDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY), initialSize);
+    return plainFallBack(new PlainLongDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY, new DirectByteBufferAllocator()), initialSize);
   }
 
   private FallbackValuesWriter<PlainIntegerDictionaryValuesWriter, PlainValuesWriter> newPlainIntegerDictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
-    return plainFallBack(new PlainIntegerDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY), initialSize);
+    return plainFallBack(new PlainIntegerDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY, new DirectByteBufferAllocator()), initialSize);
   }
 
   private FallbackValuesWriter<PlainDoubleDictionaryValuesWriter, PlainValuesWriter> newPlainDoubleDictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
-    return plainFallBack(new PlainDoubleDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY), initialSize);
+    return plainFallBack(new PlainDoubleDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY, new DirectByteBufferAllocator()), initialSize);
   }
 
   private FallbackValuesWriter<PlainFloatDictionaryValuesWriter, PlainValuesWriter> newPlainFloatDictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) {
-    return plainFallBack(new PlainFloatDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY), initialSize);
+    return plainFallBack(new PlainFloatDictionaryValuesWriter(maxDictionaryByteSize, PLAIN_DICTIONARY, PLAIN_DICTIONARY, new DirectByteBufferAllocator()), initialSize);
   }
 
   @Test
@@ -116,7 +118,7 @@ public class TestDictionary {
 
     //Fallbacked to Plain encoding, therefore use PlainValuesReader to read it back
     ValuesReader reader = new BinaryPlainValuesReader();
-    reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+    reader.initFromPage(100, cw.getBytes().toByteBuffer(), 0);
 
     for (long i = 0; i < 100; i++) {
       assertEquals(Binary.fromString("str" + i), reader.readBytes());
@@ -202,13 +204,13 @@ public class TestDictionary {
 
     DictionaryValuesReader cr = initDicReader(cw, PrimitiveTypeName.INT64);
 
-    cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
+    cr.initFromPage(COUNT, bytes1.toByteBuffer(), 0);
     for (long i = 0; i < COUNT; i++) {
       long back = cr.readLong();
       assertEquals(i % 50, back);
     }
 
-    cr.initFromPage(COUNT2, bytes2.toByteArray(), 0);
+    cr.initFromPage(COUNT2, bytes2.toByteBuffer(), 0);
     for (long i = COUNT2; i > 0; i--) {
       long back = cr.readLong();
       assertEquals(i % 50, back);
@@ -226,7 +228,7 @@ public class TestDictionary {
       }
     }
 
-    reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+    reader.initFromPage(100, cw.getBytes().toByteBuffer(), 0);
 
     for (long i = 0; i < 100; i++) {
       assertEquals(i, reader.readLong());
@@ -272,13 +274,13 @@ public class TestDictionary {
 
     final DictionaryValuesReader cr = initDicReader(cw, DOUBLE);
 
-    cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
+    cr.initFromPage(COUNT, bytes1.toByteBuffer(), 0);
     for (double i = 0; i < COUNT; i++) {
       double back = cr.readDouble();
       assertEquals(i % 50, back, 0.0);
     }
 
-    cr.initFromPage(COUNT2, bytes2.toByteArray(), 0);
+    cr.initFromPage(COUNT2, bytes2.toByteBuffer(), 0);
     for (double i = COUNT2; i > 0; i--) {
       double back = cr.readDouble();
       assertEquals(i % 50, back, 0.0);
@@ -297,7 +299,7 @@ public class TestDictionary {
       }
     }
 
-    reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+    reader.initFromPage(100, cw.getBytes().toByteBuffer(), 0);
 
     for (double i = 0; i < 100; i++) {
       assertEquals(i, reader.readDouble(), 0.00001);
@@ -343,13 +345,13 @@ public class TestDictionary {
 
     DictionaryValuesReader cr = initDicReader(cw, INT32);
 
-    cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
+    cr.initFromPage(COUNT, bytes1.toByteBuffer(), 0);
     for (int i = 0; i < COUNT; i++) {
       int back = cr.readInteger();
       assertEquals(i % 50, back);
     }
 
-    cr.initFromPage(COUNT2, bytes2.toByteArray(), 0);
+    cr.initFromPage(COUNT2, bytes2.toByteBuffer(), 0);
     for (int i = COUNT2; i > 0; i--) {
       int back = cr.readInteger();
       assertEquals(i % 50, back);
@@ -368,7 +370,7 @@ public class TestDictionary {
       }
     }
 
-    reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+    reader.initFromPage(100, cw.getBytes().toByteBuffer(), 0);
 
     for (int i = 0; i < 100; i++) {
       assertEquals(i, reader.readInteger());
@@ -414,13 +416,13 @@ public class TestDictionary {
 
     DictionaryValuesReader cr = initDicReader(cw, FLOAT);
 
-    cr.initFromPage(COUNT, bytes1.toByteArray(), 0);
+    cr.initFromPage(COUNT, bytes1.toByteBuffer(), 0);
     for (float i = 0; i < COUNT; i++) {
       float back = cr.readFloat();
       assertEquals(i % 50, back, 0.0f);
     }
 
-    cr.initFromPage(COUNT2, bytes2.toByteArray(), 0);
+    cr.initFromPage(COUNT2, bytes2.toByteBuffer(), 0);
     for (float i = COUNT2; i > 0; i--) {
       float back = cr.readFloat();
       assertEquals(i % 50, back, 0.0f);
@@ -439,7 +441,7 @@ public class TestDictionary {
       }
     }
 
-    reader.initFromPage(100, cw.getBytes().toByteArray(), 0);
+    reader.initFromPage(100, cw.getBytes().toByteBuffer(), 0);
 
     for (float i = 0; i < 100; i++) {
       assertEquals(i, reader.readFloat(), 0.00001);
@@ -473,14 +475,14 @@ public class TestDictionary {
     DictionaryValuesReader reader = initDicReader(cw, INT32);
 
     // pretend there are 100 nulls. what matters is offset = bytes.length.
-    byte[] bytes = {0x00, 0x01, 0x02, 0x03}; // data doesn't matter
-    int offset = bytes.length;
+    ByteBuffer bytes = ByteBuffer.wrap(new byte[] {0x00, 0x01, 0x02, 0x03}); // data doesn't matter
+    int offset = bytes.remaining();
     reader.initFromPage(100, bytes, offset);
   }
 
   private DictionaryValuesReader initDicReader(ValuesWriter cw, PrimitiveTypeName type)
       throws IOException {
-    final DictionaryPage dictionaryPage = cw.createDictionaryPage().copy();
+    final DictionaryPage dictionaryPage = cw.toDictPageAndClose().copy();
     final ColumnDescriptor descriptor = new ColumnDescriptor(new String[] {"foo"}, type, 0, 0);
     final Dictionary dictionary = PLAIN.initDictionary(descriptor, dictionaryPage);
     final DictionaryValuesReader cr = new DictionaryValuesReader(dictionary);
@@ -488,14 +490,14 @@ public class TestDictionary {
   }
 
   private void checkDistinct(int COUNT, BytesInput bytes, ValuesReader cr, String prefix) throws IOException {
-    cr.initFromPage(COUNT, bytes.toByteArray(), 0);
+    cr.initFromPage(COUNT, bytes.toByteBuffer(), 0);
     for (int i = 0; i < COUNT; i++) {
       Assert.assertEquals(prefix + i, cr.readBytes().toStringUsingUTF8());
     }
   }
 
   private void checkRepeated(int COUNT, BytesInput bytes, ValuesReader cr, String prefix) throws IOException {
-    cr.initFromPage(COUNT, bytes.toByteArray(), 0);
+    cr.initFromPage(COUNT, bytes.toByteBuffer(), 0);
     for (int i = 0; i < COUNT; i++) {
       Assert.assertEquals(prefix + i % 10, cr.readBytes().toStringUsingUTF8());
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java b/parquet-column/src/test/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
index 707a507..712fb27 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridIntegrationTest.java
@@ -18,9 +18,11 @@
  */
 package org.apache.parquet.column.values.rle;
 
-import java.io.ByteArrayInputStream;
+import java.nio.ByteBuffer;
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.junit.Test;
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 
 import static org.junit.Assert.assertEquals;
 
@@ -39,7 +41,7 @@ public class RunLengthBitPackingHybridIntegrationTest {
   private void doIntegrationTest(int bitWidth) throws Exception {
     long modValue = 1L << bitWidth;
 
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(bitWidth, 1000, 64000);
+    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(bitWidth, 1000, 64000, new DirectByteBufferAllocator());
     int numValues = 0;
 
     for (int i = 0; i < 100; i++) {
@@ -69,8 +71,8 @@ public class RunLengthBitPackingHybridIntegrationTest {
     }
     numValues += 1000;
 
-    byte[] encodedBytes = encoder.toBytes().toByteArray();
-    ByteArrayInputStream in = new ByteArrayInputStream(encodedBytes);
+    ByteBuffer encodedBytes = encoder.toBytes().toByteBuffer();
+    ByteBufferInputStream in = new ByteBufferInputStream(encodedBytes);
 
     RunLengthBitPackingHybridDecoder decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java b/parquet-column/src/test/java/org/apache/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
index 06664de..5696d7b 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/rle/TestRunLengthBitPackingHybridEncoder.java
@@ -21,12 +21,15 @@ package org.apache.parquet.column.values.rle;
 import static org.junit.Assert.assertEquals;
 
 import java.io.ByteArrayInputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
 import org.junit.Test;
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.DirectByteBufferAllocator;
 import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.values.bitpacking.BytePacker;
 import org.apache.parquet.column.values.bitpacking.Packer;
@@ -36,9 +39,19 @@ import org.apache.parquet.column.values.bitpacking.Packer;
  */
 public class TestRunLengthBitPackingHybridEncoder {
 
+  private RunLengthBitPackingHybridEncoder getRunLengthBitPackingHybridEncoder() {
+    return getRunLengthBitPackingHybridEncoder(3, 5, 10);
+  }
+
+  private RunLengthBitPackingHybridEncoder getRunLengthBitPackingHybridEncoder(
+      int bitWidth, int initialCapacity, int pageSize) {
+    return new RunLengthBitPackingHybridEncoder(bitWidth, initialCapacity,
+        pageSize, new DirectByteBufferAllocator());
+  }
+
   @Test
   public void testRLEOnly() throws Exception {
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5, 10);
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder();
     for (int i = 0; i < 100; i++) {
       encoder.writeInt(4);
     }
@@ -68,7 +81,7 @@ public class TestRunLengthBitPackingHybridEncoder {
     // make sure that repeated 0s at the beginning
     // of the stream don't trip up the repeat count
 
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5, 10);
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder();
     for (int i = 0; i < 10; i++) {
       encoder.writeInt(0);
     }
@@ -86,7 +99,7 @@ public class TestRunLengthBitPackingHybridEncoder {
 
   @Test
   public void testBitWidthZero() throws Exception {
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(0, 5, 10);
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder(0, 5, 10);
     for (int i = 0; i < 10; i++) {
       encoder.writeInt(0);
     }
@@ -102,8 +115,7 @@ public class TestRunLengthBitPackingHybridEncoder {
 
   @Test
   public void testBitPackingOnly() throws Exception {
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5, 10);
-
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder();
     for (int i = 0; i < 100; i++) {
       encoder.writeInt(i % 3);
     }
@@ -125,7 +137,7 @@ public class TestRunLengthBitPackingHybridEncoder {
 
   @Test
   public void testBitPackingOverflow() throws Exception {
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5, 10);
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder();
 
     for (int i = 0; i < 1000; i++) {
       encoder.writeInt(i % 3);
@@ -157,7 +169,7 @@ public class TestRunLengthBitPackingHybridEncoder {
 
   @Test
   public void testTransitionFromBitPackingToRle() throws Exception {
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(3, 5, 10);
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder();
 
     // 5 obviously bit-packed values
     encoder.writeInt(0);
@@ -195,7 +207,7 @@ public class TestRunLengthBitPackingHybridEncoder {
 
   @Test
   public void testPaddingZerosOnUnfinishedBitPackedRuns() throws Exception {
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(5, 5, 10);
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder(5, 5, 10);
     for (int i = 0; i < 9; i++) {
       encoder.writeInt(i+1);
     }
@@ -214,7 +226,7 @@ public class TestRunLengthBitPackingHybridEncoder {
 
   @Test
   public void testSwitchingModes() throws Exception {
-    RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(9, 100, 1000);
+    RunLengthBitPackingHybridEncoder encoder = getRunLengthBitPackingHybridEncoder(9, 100, 1000);
 
     // rle first
     for (int i = 0; i < 25; i++) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java b/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java
index 3abf804..aff3937 100644
--- a/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/io/PerfTest.java
@@ -27,6 +27,7 @@ import static org.apache.parquet.example.Paper.schema3;
 import java.util.logging.Level;
 
 import org.apache.parquet.Log;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import org.apache.parquet.column.ParquetProperties.WriterVersion;
 import org.apache.parquet.column.impl.ColumnWriteStoreV1;
 import org.apache.parquet.column.page.mem.MemPageStore;
@@ -77,7 +78,7 @@ public class PerfTest {
 
 
   private static void write(MemPageStore memPageStore) {
-    ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 50*1024*1024, 50*1024*1024, false, WriterVersion.PARQUET_1_0);
+    ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 50*1024*1024, 50*1024*1024, false, WriterVersion.PARQUET_1_0, new HeapByteBufferAllocator());
     MessageColumnIO columnIO = newColumnFactory(schema);
 
     GroupWriter groupWriter = new GroupWriter(columnIO.getRecordWriter(columns), schema);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java b/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java
index e7274cc..06f22b6 100644
--- a/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java
+++ b/parquet-column/src/test/java/org/apache/parquet/io/TestColumnIO.java
@@ -38,6 +38,7 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -526,7 +527,7 @@ public class TestColumnIO {
   }
 
   private ColumnWriteStoreV1 newColumnWriteStore(MemPageStore memPageStore) {
-    return new ColumnWriteStoreV1(memPageStore, 800, 800, useDictionary, WriterVersion.PARQUET_1_0);
+    return new ColumnWriteStoreV1(memPageStore, 800, 800, useDictionary, WriterVersion.PARQUET_1_0, new HeapByteBufferAllocator());
   }
 
   @Test
@@ -599,6 +600,8 @@ public class TestColumnIO {
     groupWriter.write(r2);
     recordWriter.flush();
     columns.validate();
+    columns.flush();
+    columns.close();
   }
 }
 final class ValidatingColumnWriteStore implements ColumnWriteStore {
@@ -610,6 +613,11 @@ final class ValidatingColumnWriteStore implements ColumnWriteStore {
   }
 
   @Override
+  public void close() {
+
+  }
+
+  @Override
   public ColumnWriter getColumnWriter(final ColumnDescriptor path) {
     return new ColumnWriter() {
       private void validate(Object value, int repetitionLevel,
@@ -630,6 +638,11 @@ final class ValidatingColumnWriteStore implements ColumnWriteStore {
       }
 
       @Override
+      public void write(float value, int repetitionLevel, int definitionLevel) {
+        validate(value, repetitionLevel, definitionLevel);
+      }
+
+      @Override
       public void write(boolean value, int repetitionLevel, int definitionLevel) {
         validate(value, repetitionLevel, definitionLevel);
       }
@@ -645,8 +658,13 @@ final class ValidatingColumnWriteStore implements ColumnWriteStore {
       }
 
       @Override
-      public void write(float value, int repetitionLevel, int definitionLevel) {
-        validate(value, repetitionLevel, definitionLevel);
+      public void close() {
+
+      }
+
+      @Override
+      public long getBufferedSizeInMemory() {
+        throw new UnsupportedOperationException();
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/io/TestFiltered.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/io/TestFiltered.java b/parquet-column/src/test/java/org/apache/parquet/io/TestFiltered.java
index 9fde4b1..25b629b 100644
--- a/parquet-column/src/test/java/org/apache/parquet/io/TestFiltered.java
+++ b/parquet-column/src/test/java/org/apache/parquet/io/TestFiltered.java
@@ -21,6 +21,7 @@ package org.apache.parquet.io;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
 import org.apache.parquet.io.api.RecordConsumer;
 import org.junit.Test;
 
@@ -258,7 +259,7 @@ public class TestFiltered {
 
   private MemPageStore writeTestRecords(MessageColumnIO columnIO, int number) {
     MemPageStore memPageStore = new MemPageStore(number * 2);
-    ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 800, 800, false, WriterVersion.PARQUET_1_0);
+    ColumnWriteStoreV1 columns = new ColumnWriteStoreV1(memPageStore, 800, 800, false, WriterVersion.PARQUET_1_0, new HeapByteBufferAllocator());
 
     RecordConsumer recordWriter = columnIO.getRecordWriter(columns);
     GroupWriter groupWriter = new GroupWriter(recordWriter, schema);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/test/java/org/apache/parquet/io/api/TestBinary.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/io/api/TestBinary.java b/parquet-column/src/test/java/org/apache/parquet/io/api/TestBinary.java
index bd8a69d..c8444dc 100644
--- a/parquet-column/src/test/java/org/apache/parquet/io/api/TestBinary.java
+++ b/parquet-column/src/test/java/org/apache/parquet/io/api/TestBinary.java
@@ -18,6 +18,8 @@
  */
 package org.apache.parquet.io.api;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
@@ -143,6 +145,29 @@ public class TestBinary {
   }
 
   @Test
+  public void testEqualityMethods() throws Exception {
+    Binary bin1 = Binary.fromConstantByteArray("alice".getBytes(), 1, 3);
+    Binary bin2 = Binary.fromConstantByteBuffer(ByteBuffer.wrap("alice".getBytes(), 1, 3));
+    assertEquals(bin1, bin2);
+  }
+
+  @Test
+  public void testWriteAllTo() throws Exception {
+    byte[] orig = {10, 9 ,8, 7, 6, 5, 4, 3, 2, 1};
+    testWriteAllToHelper(Binary.fromConstantByteBuffer(ByteBuffer.wrap(orig)), orig);
+    ByteBuffer buf = ByteBuffer.allocateDirect(orig.length);
+    buf.put(orig);
+    buf.flip();
+    testWriteAllToHelper(Binary.fromConstantByteBuffer(buf), orig);
+  }
+
+  private void testWriteAllToHelper(Binary binary, byte[] orig) throws IOException {
+    ByteArrayOutputStream out = new ByteArrayOutputStream(orig.length);
+    binary.writeTo(out);
+    assertArrayEquals(orig, out.toByteArray());
+  }
+
+  @Test
   public void testFromStringBinary() throws Exception {
     testBinary(STRING_BF, false);
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/IOExceptionUtils.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/IOExceptionUtils.java b/parquet-common/src/main/java/org/apache/parquet/IOExceptionUtils.java
new file mode 100644
index 0000000..2ac8a2b
--- /dev/null
+++ b/parquet-common/src/main/java/org/apache/parquet/IOExceptionUtils.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Utilities for managing I/O resources.
+ */
+public class IOExceptionUtils {
+
+	/**
+	 * Call the #close() method on a {@see Closable}, wrapping any IOException
+	 * in a runtime exception.
+	 *
+	 * @param closeable - resource to close
+	 */
+	public static void closeQuietly(Closeable closeable) {
+		try {
+			closeable.close();
+		} catch(IOException e) {
+			throw new ParquetRuntimeException("Error closing I/O related resources.", e) {};
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/OutputStreamCloseException.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/OutputStreamCloseException.java b/parquet-common/src/main/java/org/apache/parquet/OutputStreamCloseException.java
new file mode 100644
index 0000000..5271000
--- /dev/null
+++ b/parquet-common/src/main/java/org/apache/parquet/OutputStreamCloseException.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.parquet;
+
+/**
+ * Runtime exception indicating that a stream failed to be closed properly.
+ *
+ * Used to wrap up the checked IOException usually thrown from IO operations,
+ * these are generally not recoverable so it does not make sense to pollute the
+ * codebase declaring that they can be thrown whenever resources are being
+ * closed out.
+ */
+public class OutputStreamCloseException extends ParquetRuntimeException {
+
+  private static final long serialVersionUID = 1L;
+
+  public OutputStreamCloseException() {
+  }
+
+  public OutputStreamCloseException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public OutputStreamCloseException(String message) {
+    super(message);
+  }
+
+  public OutputStreamCloseException(Throwable cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/ParquetRuntimeException.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/ParquetRuntimeException.java b/parquet-common/src/main/java/org/apache/parquet/ParquetRuntimeException.java
index f67b15a..d0f13a8 100644
--- a/parquet-common/src/main/java/org/apache/parquet/ParquetRuntimeException.java
+++ b/parquet-common/src/main/java/org/apache/parquet/ParquetRuntimeException.java
@@ -18,6 +18,9 @@
  */
 package org.apache.parquet;
 
+import java.io.Closeable;
+import java.io.IOException;
+
 /**
  * The parent class for all runtime exceptions
  *
@@ -42,5 +45,4 @@ abstract public class ParquetRuntimeException extends RuntimeException {
   public ParquetRuntimeException(Throwable cause) {
     super(cause);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferAllocator.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferAllocator.java b/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferAllocator.java
new file mode 100644
index 0000000..ee36b74
--- /dev/null
+++ b/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferAllocator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.bytes;
+
+import java.nio.ByteBuffer;
+
+public interface ByteBufferAllocator {
+  ByteBuffer allocate(int size);
+
+  /**
+   * For RefCounted implementations using direct memory, the release method
+   * needs to be called to free references to the allocated memory.
+   */
+  void release(ByteBuffer b);
+
+  /**
+   * Indicates if this allocator will produce ByteBuffers backed by direct memory.
+   *
+   * @return true if direct memory backed buffers will be created by this allocator, else false
+   */
+  boolean isDirect();
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java b/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java
new file mode 100644
index 0000000..5b3b853
--- /dev/null
+++ b/parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.bytes;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+/**
+ * This ByteBufferInputStream does not consume the ByteBuffer being passed in, 
+ * but will create a slice of the current buffer.
+ */
+public class ByteBufferInputStream extends InputStream {
+	
+  protected ByteBuffer byteBuf;
+  protected int initPos;
+  protected int count;
+  public ByteBufferInputStream(ByteBuffer buffer) {
+    this(buffer, buffer.position(), buffer.remaining());
+  }
+  
+  public ByteBufferInputStream(ByteBuffer buffer, int offset, int count) {
+    ByteBuffer temp = buffer.duplicate();
+    temp.position(offset);
+    byteBuf = temp.slice();
+    byteBuf.limit(count);
+    this.initPos = offset;
+    this.count = count;
+  }
+  
+  public ByteBuffer toByteBuffer() {
+    return byteBuf.slice();
+  }
+  
+  @Override
+  public int read() throws IOException {
+    if (!byteBuf.hasRemaining()) {
+    	return -1;
+    }
+    //Workaround for unsigned byte
+    return byteBuf.get() & 0xFF;
+  }
+
+  @Override
+  public int read(byte[] bytes, int offset, int length) throws IOException {
+    int count = Math.min(byteBuf.remaining(), length);
+    if (count == 0) return -1;
+    byteBuf.get(bytes, offset, count);
+    return count;
+  }
+  
+  @Override
+  public long skip(long n) {
+	  if (n > byteBuf.remaining())
+	    n = byteBuf.remaining();
+	  int pos = byteBuf.position();
+	  byteBuf.position((int)(pos + n));
+	  return n;
+  }
+
+
+  @Override
+  public int available() {
+    return byteBuf.remaining();
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java b/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java
index d96a1e5..d40721a 100644
--- a/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java
+++ b/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java
@@ -22,6 +22,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 
 import org.apache.parquet.Log;
@@ -53,6 +54,21 @@ public class BytesUtils {
    * @return
    * @throws IOException
    */
+  public static int readIntLittleEndian(ByteBuffer in, int offset) throws IOException {
+    int ch4 = in.get(offset) & 0xff;
+    int ch3 = in.get(offset + 1) & 0xff;
+    int ch2 = in.get(offset + 2) & 0xff;
+    int ch1 = in.get(offset + 3) & 0xff;
+    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
+  }
+  
+  /**
+   * 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;
@@ -205,6 +221,14 @@ public class BytesUtils {
     out.write(value & 0x7F);
   }
 
+  public static void writeUnsignedVarInt(int value, ByteBuffer dest) throws IOException {
+    while ((value & 0xFFFFFF80) != 0L) {
+      dest.putInt((value & 0x7F) | 0x80);
+      value >>>= 7;
+    }
+    dest.putInt(value & 0x7F);
+  }
+
   public static void writeZigZagVarInt(int intValue, OutputStream out) throws IOException{
     writeUnsignedVarInt((intValue << 1) ^ (intValue >> 31), out);
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/bytes/DirectByteBufferAllocator.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/bytes/DirectByteBufferAllocator.java b/parquet-common/src/main/java/org/apache/parquet/bytes/DirectByteBufferAllocator.java
new file mode 100644
index 0000000..9fe4538
--- /dev/null
+++ b/parquet-common/src/main/java/org/apache/parquet/bytes/DirectByteBufferAllocator.java
@@ -0,0 +1,43 @@
+/* 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.bytes;
+
+import java.nio.ByteBuffer;
+
+public class DirectByteBufferAllocator implements ByteBufferAllocator{
+  public static final DirectByteBufferAllocator getInstance(){return new DirectByteBufferAllocator();}
+  public DirectByteBufferAllocator() {
+    super();
+  }
+
+  public ByteBuffer allocate(final int size) {
+    return ByteBuffer.allocateDirect(size);
+  }
+
+  @Override
+  public void release(ByteBuffer b) {
+    // The ByteBuffer.allocateDirect
+    return;
+  }
+
+  @Override
+  public boolean isDirect() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-common/src/main/java/org/apache/parquet/bytes/HeapByteBufferAllocator.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/org/apache/parquet/bytes/HeapByteBufferAllocator.java b/parquet-common/src/main/java/org/apache/parquet/bytes/HeapByteBufferAllocator.java
new file mode 100644
index 0000000..c5f475d
--- /dev/null
+++ b/parquet-common/src/main/java/org/apache/parquet/bytes/HeapByteBufferAllocator.java
@@ -0,0 +1,44 @@
+/* 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.bytes;
+
+import java.nio.ByteBuffer;
+
+public class HeapByteBufferAllocator implements ByteBufferAllocator{
+
+  public static final HeapByteBufferAllocator getInstance(){ return new HeapByteBufferAllocator();}
+
+  public HeapByteBufferAllocator() {
+    super();
+  }
+
+  public ByteBuffer allocate(final int size) {
+    return ByteBuffer.allocate(size);
+  }
+
+  public void release(ByteBuffer b) {
+    return;
+  }
+
+  @Override
+  public boolean isDirect() {
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java b/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java
index ac334ae..40190ee 100644
--- a/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java
@@ -25,6 +25,9 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Arrays;
 import java.util.List;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
 
 import org.apache.parquet.Log;
 
@@ -71,6 +74,15 @@ abstract public class BytesInput {
   public static BytesInput from(InputStream in, int bytes) {
     return new StreamBytesInput(in, bytes);
   }
+  
+  /**
+   * @param buffer
+   * @param length number of bytes to read
+   * @return a BytesInput that will read the given bytes from the ByteBuffer
+   */
+  public static BytesInput from(ByteBuffer buffer, int offset, int length) {
+    return new ByteBufferBytesInput(buffer, offset, length);
+  }
 
   /**
    *
@@ -121,7 +133,7 @@ abstract public class BytesInput {
   }
 
   /**
-   * @param arrayOut
+   * @param baos - stream to wrap into a BytesInput
    * @return a BytesInput that will write the content of the buffer
    */
   public static BytesInput from(ByteArrayOutputStream baos) {
@@ -166,6 +178,24 @@ abstract public class BytesInput {
 
   /**
    *
+   * @return a new ByteBuffer materializing the contents of this input
+   * @throws IOException
+   */
+  public ByteBuffer toByteBuffer() throws IOException {
+    return ByteBuffer.wrap(toByteArray());
+  }
+
+  /**
+   *
+   * @return a new InputStream materializing the contents of this input
+   * @throws IOException
+   */
+  public InputStream toInputStream() throws IOException {
+    return new ByteBufferInputStream(toByteBuffer());
+  }
+
+  /**
+   *
    * @return the size in bytes that would be written
    */
   abstract public long size();
@@ -258,6 +288,10 @@ abstract public class BytesInput {
       BytesUtils.writeIntLittleEndian(out, intValue);
     }
 
+    public ByteBuffer toByteBuffer() throws IOException {
+      return ByteBuffer.allocate(4).putInt(0, intValue);
+    }
+
     @Override
     public long size() {
       return 4;
@@ -278,6 +312,12 @@ abstract public class BytesInput {
       BytesUtils.writeUnsignedVarInt(intValue, out);
     }
 
+    public ByteBuffer toByteBuffer() throws IOException {
+      ByteBuffer ret = ByteBuffer.allocate((int) size());
+      BytesUtils.writeUnsignedVarInt(intValue, ret);
+      return ret;
+    }
+
     @Override
     public long size() {
       int s = 5 - ((Integer.numberOfLeadingZeros(intValue) + 3) / 7);
@@ -296,6 +336,10 @@ abstract public class BytesInput {
       return 0;
     }
 
+    public ByteBuffer toByteBuffer() throws IOException {
+      return ByteBuffer.allocate(0);
+    }
+
   }
 
   private static class CapacityBAOSBytesInput extends BytesInput {
@@ -355,11 +399,49 @@ abstract public class BytesInput {
       out.write(in, offset, length);
     }
 
+    public ByteBuffer toByteBuffer() throws IOException {
+      return ByteBuffer.wrap(in, offset, length);
+    }
+
     @Override
     public long size() {
       return length;
     }
 
   }
+  
+  private static class ByteBufferBytesInput extends BytesInput {
+    
+    private final ByteBuffer byteBuf;
+    private final int length;
+    private final int offset;
 
+    private ByteBufferBytesInput(ByteBuffer byteBuf, int offset, int length) {
+      this.byteBuf = byteBuf;
+      this.offset = offset;
+      this.length = length;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      final WritableByteChannel outputChannel = Channels.newChannel(out);
+      byteBuf.position(offset);
+      ByteBuffer tempBuf = byteBuf.slice();
+      tempBuf.limit(length);
+      outputChannel.write(tempBuf);
+    }
+    
+    @Override
+    public ByteBuffer toByteBuffer() throws IOException {
+      byteBuf.position(offset);
+      ByteBuffer buf = byteBuf.slice();
+      buf.limit(length);
+      return buf;
+    }
+
+    @Override
+    public long size() {
+      return length;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java b/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java
index 1670f9c..6155565 100644
--- a/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java
@@ -21,16 +21,17 @@ package org.apache.parquet.bytes;
 import static java.lang.Math.max;
 import static java.lang.Math.pow;
 import static java.lang.String.format;
-import static java.lang.System.arraycopy;
 import static org.apache.parquet.Preconditions.checkArgument;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.parquet.Log;
+import org.apache.parquet.OutputStreamCloseException;
 
 /**
  * Similar to a {@link ByteArrayOutputStream}, but uses a different strategy for growing that does not involve copying.
@@ -54,16 +55,17 @@ import org.apache.parquet.Log;
  */
 public class CapacityByteArrayOutputStream extends OutputStream {
   private static final Log LOG = Log.getLog(CapacityByteArrayOutputStream.class);
-  private static final byte[] EMPTY_SLAB = new byte[0];
+  private static final ByteBuffer EMPTY_SLAB = ByteBuffer.wrap(new byte[0]);
 
   private int initialSlabSize;
   private final int maxCapacityHint;
-  private final List<byte[]> slabs = new ArrayList<byte[]>();
+  private final List<ByteBuffer> slabs = new ArrayList<ByteBuffer>();
 
-  private byte[] currentSlab;
+  private ByteBuffer currentSlab;
   private int currentSlabIndex;
   private int bytesAllocated = 0;
   private int bytesUsed = 0;
+  private ByteBufferAllocator allocator;
 
   /**
    * Return an initial slab size such that a CapacityByteArrayOutputStream constructed with it
@@ -86,38 +88,64 @@ public class CapacityByteArrayOutputStream extends OutputStream {
     return max(minSlabSize, ((int) (targetCapacity / pow(2, targetNumSlabs))));
   }
 
+  public static CapacityByteArrayOutputStream withTargetNumSlabs(
+      int minSlabSize, int maxCapacityHint, int targetNumSlabs) {
+    return withTargetNumSlabs(minSlabSize, maxCapacityHint, targetNumSlabs, new HeapByteBufferAllocator());
+  }
+
   /**
    * Construct a CapacityByteArrayOutputStream configured such that its initial slab size is
    * determined by {@link #initialSlabSizeHeuristic}, with targetCapacity == maxCapacityHint
    */
   public static CapacityByteArrayOutputStream withTargetNumSlabs(
-      int minSlabSize, int maxCapacityHint, int targetNumSlabs) {
+      int minSlabSize, int maxCapacityHint, int targetNumSlabs, ByteBufferAllocator allocator) {
 
     return new CapacityByteArrayOutputStream(
         initialSlabSizeHeuristic(minSlabSize, maxCapacityHint, targetNumSlabs),
-        maxCapacityHint);
+        maxCapacityHint, allocator);
   }
 
   /**
    * Defaults maxCapacityHint to 1MB
    * @param initialSlabSize
-   * @deprecated use {@link CapacityByteArrayOutputStream#CapacityByteArrayOutputStream(int, int)}
+   * @deprecated use {@link CapacityByteArrayOutputStream#CapacityByteArrayOutputStream(int, int, ByteBufferAllocator)}
    */
   @Deprecated
   public CapacityByteArrayOutputStream(int initialSlabSize) {
-    this(initialSlabSize, 1024 * 1024);
+    this(initialSlabSize, 1024 * 1024, new HeapByteBufferAllocator());
+  }
+
+  /**
+   * Defaults maxCapacityHint to 1MB
+   * @param initialSlabSize
+   * @deprecated use {@link CapacityByteArrayOutputStream#CapacityByteArrayOutputStream(int, int, ByteBufferAllocator)}
+   */
+  @Deprecated
+  public CapacityByteArrayOutputStream(int initialSlabSize, ByteBufferAllocator allocator) {
+    this(initialSlabSize, 1024 * 1024, allocator);
   }
 
   /**
    * @param initialSlabSize the size to make the first slab
    * @param maxCapacityHint a hint (not guarantee) of the max amount of data written to this stream
+   * @deprecated use {@link CapacityByteArrayOutputStream#CapacityByteArrayOutputStream(int, int, ByteBufferAllocator)}
    */
+  @Deprecated
   public CapacityByteArrayOutputStream(int initialSlabSize, int maxCapacityHint) {
+    this(initialSlabSize, maxCapacityHint, new HeapByteBufferAllocator());
+  }
+
+  /**
+   * @param initialSlabSize the size to make the first slab
+   * @param maxCapacityHint a hint (not guarantee) of the max amount of data written to this stream
+   */
+  public CapacityByteArrayOutputStream(int initialSlabSize, int maxCapacityHint, ByteBufferAllocator allocator) {
     checkArgument(initialSlabSize > 0, "initialSlabSize must be > 0");
     checkArgument(maxCapacityHint > 0, "maxCapacityHint must be > 0");
     checkArgument(maxCapacityHint >= initialSlabSize, String.format("maxCapacityHint can't be less than initialSlabSize %d %d", initialSlabSize, maxCapacityHint));
     this.initialSlabSize = initialSlabSize;
     this.maxCapacityHint = maxCapacityHint;
+    this.allocator = allocator;
     reset();
   }
 
@@ -145,7 +173,7 @@ public class CapacityByteArrayOutputStream extends OutputStream {
 
     if (Log.DEBUG) LOG.debug(format("used %d slabs, adding new slab of size %d", slabs.size(), nextSlabSize));
 
-    this.currentSlab = new byte[nextSlabSize];
+    this.currentSlab = allocator.allocate(nextSlabSize);
     this.slabs.add(currentSlab);
     this.bytesAllocated += nextSlabSize;
     this.currentSlabIndex = 0;
@@ -153,11 +181,12 @@ public class CapacityByteArrayOutputStream extends OutputStream {
 
   @Override
   public void write(int b) {
-    if (currentSlabIndex == currentSlab.length) {
+    if (!currentSlab.hasRemaining()) {
       addSlab(1);
     }
-    currentSlab[currentSlabIndex] = (byte) b;
+    currentSlab.put(currentSlabIndex, (byte) b);
     currentSlabIndex += 1;
+    currentSlab.position(currentSlabIndex);
     bytesUsed += 1;
   }
 
@@ -168,18 +197,34 @@ public class CapacityByteArrayOutputStream extends OutputStream {
       throw new IndexOutOfBoundsException(
           String.format("Given byte array of size %d, with requested length(%d) and offset(%d)", b.length, len, off));
     }
-    if (currentSlabIndex + len >= currentSlab.length) {
-      final int length1 = currentSlab.length - currentSlabIndex;
-      arraycopy(b, off, currentSlab, currentSlabIndex, length1);
+    if (len >= currentSlab.remaining()) {
+      final int length1 = currentSlab.remaining();
+      currentSlab.put(b, off, length1);
+      bytesUsed += length1;
+      currentSlabIndex += length1;
       final int length2 = len - length1;
       addSlab(length2);
-      arraycopy(b, off + length1, currentSlab, currentSlabIndex, length2);
+      currentSlab.put(b, off + length1, length2);
       currentSlabIndex = length2;
+      bytesUsed += length2;
     } else {
-      arraycopy(b, off, currentSlab, currentSlabIndex, len);
+      currentSlab.put(b, off, len);
       currentSlabIndex += len;
+      bytesUsed += len;
+    }
+  }
+
+  private void writeToOutput(OutputStream out, ByteBuffer buf, int len) throws IOException {
+    if (buf.hasArray()) {
+      out.write(buf.array(), buf.arrayOffset(), len);
+    } else {
+      // The OutputStream interface only takes a byte[], unfortunately this means that a ByteBuffer
+      // not backed by a byte array must be copied to fulfil this interface
+      byte[] copy = new byte[len];
+      buf.flip();
+      buf.get(copy);
+      out.write(copy);
     }
-    bytesUsed += len;
   }
 
   /**
@@ -191,10 +236,9 @@ public class CapacityByteArrayOutputStream extends OutputStream {
    */
   public void writeTo(OutputStream out) throws IOException {
     for (int i = 0; i < slabs.size() - 1; i++) {
-      final byte[] slab = slabs.get(i);
-      out.write(slab);
+      writeToOutput(out, slabs.get(i), slabs.get(i).position());
     }
-    out.write(currentSlab, 0, currentSlabIndex);
+    writeToOutput(out, currentSlab, currentSlabIndex);
   }
 
   /**
@@ -222,6 +266,9 @@ public class CapacityByteArrayOutputStream extends OutputStream {
     // 7 = 2^3 - 1 so that doubling the initial size 3 times will get to the same size
     this.initialSlabSize = max(bytesUsed / 7, initialSlabSize);
     if (Log.DEBUG) LOG.debug(String.format("initial slab of size %d", initialSlabSize));
+    for (ByteBuffer slab : slabs) {
+      allocator.release(slab);
+    }
     this.slabs.clear();
     this.bytesAllocated = 0;
     this.bytesUsed = 0;
@@ -249,13 +296,13 @@ public class CapacityByteArrayOutputStream extends OutputStream {
 
     long seen = 0;
     for (int i = 0; i < slabs.size(); i++) {
-      byte[] slab = slabs.get(i);
-      if (index < seen + slab.length) {
+      ByteBuffer slab = slabs.get(i);
+      if (index < seen + slab.limit()) {
         // ok found index
-        slab[(int)(index-seen)] = value;
+        slab.put((int)(index-seen), value);
         break;
       }
-      seen += slab.length;
+      seen += slab.limit();
     }
   }
 
@@ -273,4 +320,16 @@ public class CapacityByteArrayOutputStream extends OutputStream {
   int getSlabCount() {
     return slabs.size();
   }
+
+  @Override
+  public void close() {
+    for (ByteBuffer slab : slabs) {
+      allocator.release(slab);
+    }
+    try {
+      super.close();
+    }catch(IOException e){
+      throw new OutputStreamCloseException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java b/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java
index da4e92f..9d4a8a9 100644
--- a/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java
@@ -18,6 +18,9 @@
  */
 package org.apache.parquet.bytes;
 
+import org.apache.parquet.IOExceptionUtils;
+import org.apache.parquet.ParquetRuntimeException;
+
 import java.io.IOException;
 import java.io.OutputStream;
 
@@ -210,4 +213,8 @@ public class LittleEndianDataOutputStream extends OutputStream {
     writeLong(Double.doubleToLongBits(v));
   }
 
+  public void close() {
+    IOExceptionUtils.closeQuietly(out);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java b/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java
index b9a37ad..675576c 100644
--- a/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java
+++ b/parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java
@@ -18,6 +18,8 @@
  */
 package org.apache.parquet.column.values.bitpacking;
 
+import java.nio.ByteBuffer;
+
 /**
  * Packs and unpacks into bytes
  *
@@ -71,7 +73,15 @@ public abstract class BytePacker {
    * @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);
+  public abstract void unpack8Values(final ByteBuffer input, final int inPos, final int[] output, final int outPos);
+
+  /**
+   * Compatibility API
+   */
+  @Deprecated
+  public void unpack8Values(final byte[] input, final int inPos, final int[] output, final int outPos) {
+    unpack8Values(ByteBuffer.wrap(input), inPos, output, outPos);
+  }
 
   /**
    * unpack bitWidth * 4 bytes from input at inPos into 32 values in output at outPos.
@@ -81,6 +91,13 @@ public abstract class BytePacker {
    * @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);
+  public abstract void unpack32Values(ByteBuffer input, int inPos, int[] output, int outPos);
 
+  /**
+   * Compatibility API
+   */
+  @Deprecated
+  public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
+    unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
+  }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestBytesInput.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestBytesInput.java b/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestBytesInput.java
new file mode 100644
index 0000000..1cb0304
--- /dev/null
+++ b/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestBytesInput.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.bytes;
+
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestBytesInput {
+
+	@Test
+	public void testWriteInt() throws Throwable {
+		int[] testVals = {
+				Integer.MIN_VALUE,
+				Integer.MAX_VALUE,
+				0, 100, 1000, 0xdaedbeef};
+		for (Integer testVal : testVals) {
+			BytesInput varInt = BytesInput.fromUnsignedVarInt(testVal);
+			byte[] rno = varInt.toByteArray();
+			int i = BytesUtils.readUnsignedVarInt(new ByteArrayInputStream(rno));
+			assertEquals((int) testVal, i);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestCapacityByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestCapacityByteArrayOutputStream.java b/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestCapacityByteArrayOutputStream.java
index b80fe40..89db198 100644
--- a/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestCapacityByteArrayOutputStream.java
+++ b/parquet-encoding/src/test/java/org/apache/parquet/bytes/TestCapacityByteArrayOutputStream.java
@@ -63,7 +63,7 @@ public class TestCapacityByteArrayOutputStream {
   }
 
   protected CapacityByteArrayOutputStream newCapacityBAOS(int initialSize) {
-    return new CapacityByteArrayOutputStream(10, 1000000);
+    return new CapacityByteArrayOutputStream(initialSize, 1000000, new HeapByteBufferAllocator());
   }
 
   @Test
@@ -129,12 +129,12 @@ public class TestCapacityByteArrayOutputStream {
       assertEquals(i % (v * 3), byteArray[i]);
     }
     // verifying we have not created 500 * 23 / 10 slabs
-    assertTrue("slab count: " + capacityByteArrayOutputStream.getSlabCount(),capacityByteArrayOutputStream.getSlabCount() <= 20);
+    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);
+    assertTrue("slab count: " + capacityByteArrayOutputStream.getSlabCount(), capacityByteArrayOutputStream.getSlabCount() <= 2);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java
index a5ce37e..8df5f39 100644
--- a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java
+++ b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking.java
@@ -21,6 +21,7 @@ package org.apache.parquet.column.values.bitpacking;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -50,7 +51,7 @@ public class TestByteBitPacking {
     byte[] packed = new byte[packer.getBitWidth() * 4];
     packer.pack32Values(values, 0, packed, 0);
     LOG.debug("packed: " + TestBitPacking.toString(packed));
-    packer.unpack32Values(packed, 0, unpacked, 0);
+    packer.unpack32Values(ByteBuffer.wrap(packed), 0, unpacked, 0);
   }
 
   private int[] generateValues(int bitWidth) {
@@ -140,7 +141,7 @@ public class TestByteBitPacking {
         LOG.debug("Gener. out: " + TestBitPacking.toString(packedGenerated));
         Assert.assertEquals(pack.name() + " width " + i, TestBitPacking.toString(packedByLemireAsBytes), TestBitPacking.toString(packedGenerated));
 
-        bytePacker.unpack32Values(packedByLemireAsBytes, 0, unpacked, 0);
+        bytePacker.unpack32Values(ByteBuffer.wrap(packedByLemireAsBytes), 0, unpacked, 0);
         LOG.debug("Output: " + TestBitPacking.toString(unpacked));
 
         Assert.assertArrayEquals("width " + i, values, unpacked);

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java
index e0c97e0..2c5fa58 100644
--- a/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java
+++ b/parquet-encoding/src/test/java/org/apache/parquet/column/values/bitpacking/TestLemireBitPacking.java
@@ -21,6 +21,7 @@ package org.apache.parquet.column.values.bitpacking;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -64,7 +65,7 @@ public class TestLemireBitPacking {
   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);
+    packer.unpack32Values(ByteBuffer.wrap(packed), 0, unpacked, 0);
   }
 
   private int[] generateValues(int bitWidth) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-generator/src/main/java/org/apache/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java
----------------------------------------------------------------------
diff --git a/parquet-generator/src/main/java/org/apache/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java b/parquet-generator/src/main/java/org/apache/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java
index 9a7c562..3d182e2 100644
--- a/parquet-generator/src/main/java/org/apache/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java
+++ b/parquet-generator/src/main/java/org/apache/parquet/encoding/bitpacking/ByteBasedBitPackingGenerator.java
@@ -50,6 +50,7 @@ public class ByteBasedBitPackingGenerator {
     }
     FileWriter fw = new FileWriter(file);
     fw.append("package org.apache.parquet.column.values.bitpacking;\n");
+    fw.append("import java.nio.ByteBuffer;\n");
     fw.append("\n");
     fw.append("/**\n");
     if (msbFirst) {
@@ -97,8 +98,10 @@ public class ByteBasedBitPackingGenerator {
     generatePack(fw, bitWidth, 4, msbFirst);
 
     // Unpacking
-    generateUnpack(fw, bitWidth, 1, msbFirst);
-    generateUnpack(fw, bitWidth, 4, msbFirst);
+    generateUnpack(fw, bitWidth, 1, msbFirst, true);
+    generateUnpack(fw, bitWidth, 1, msbFirst, false);
+    generateUnpack(fw, bitWidth, 4, msbFirst, true);
+    generateUnpack(fw, bitWidth, 4, msbFirst, false);
 
     fw.append("  }\n");
   }
@@ -203,9 +206,15 @@ public class ByteBasedBitPackingGenerator {
     fw.append("    }\n");
   }
 
-  private static void generateUnpack(FileWriter fw, int bitWidth, int batch, boolean msbFirst)
+  private static void generateUnpack(FileWriter fw, int bitWidth, int batch, boolean msbFirst, boolean useByteArray)
       throws IOException {
-    fw.append("    public final void unpack" + (batch * 8) + "Values(final byte[] in, final int inPos, final int[] out, final int outPos) {\n");
+    final String bufferDataType;
+    if (useByteArray) {
+      bufferDataType = "byte[]";
+    } else {
+      bufferDataType = "ByteBuffer";
+    }
+    fw.append("    public final void unpack" + (batch * 8) + "Values(final " + bufferDataType + " 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) {
@@ -228,7 +237,14 @@ public class ByteBasedBitPackingGenerator {
           } else if (shift > 0){
             shiftString = "<<  " + shift;
           }
-          fw.append(" (((((int)in[" + align(byteIndex, 2) + " + inPos]) & 255) " + shiftString + ") & " + mask + ")");
+          final String byteAccess;
+          if (useByteArray) {
+            byteAccess = "in[" + align(byteIndex, 2) + " + inPos]";
+          } else {
+            // use ByteBuffer#get(index) method
+            byteAccess = "in.get(" + align(byteIndex, 2) + " + inPos)";
+          }
+          fw.append(" (((((int)" + byteAccess + ") & 255) " + shiftString + ") & " + mask + ")");
         }
         fw.append(";\n");
       }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml
index a7f9d2c..2f2e932 100644
--- a/parquet-hadoop/pom.xml
+++ b/parquet-hadoop/pom.xml
@@ -92,6 +92,11 @@
       <version>1.9.5</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>commons-pool</groupId>
+      <artifactId>commons-pool</artifactId>
+      <version>1.5.4</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
index fdeb2ba..6821bbf 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
@@ -36,6 +36,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.parquet.CorruptStatistics;
 import org.apache.parquet.Log;
 import org.apache.parquet.hadoop.metadata.ColumnPath;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java
index 6840950..8bf882f 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *   http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@@ -18,14 +18,14 @@
  */
 package org.apache.parquet.hadoop;
 
-import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -34,18 +34,64 @@ import org.apache.hadoop.io.compress.Compressor;
 import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 
-class CodecFactory {
+public class CodecFactory {
+
+  protected static final Map<String, CompressionCodec> CODEC_BY_NAME = Collections
+      .synchronizedMap(new HashMap<String, CompressionCodec>());
+
+  private final Map<CompressionCodecName, BytesCompressor> compressors = new HashMap<CompressionCodecName, BytesCompressor>();
+  private final Map<CompressionCodecName, BytesDecompressor> decompressors = new HashMap<CompressionCodecName, BytesDecompressor>();
 
-  public class BytesDecompressor {
+  protected final Configuration configuration;
+  protected final int pageSize;
+
+  /**
+   * Create a new codec factory.
+   *
+   * @param configuration used to pass compression codec configuration information
+   * @param pageSize the expected page size, does not set a hard limit, currently just
+   *                 used to set the initial size of the output stream used when
+   *                 compressing a buffer. If this factory is only used to construct
+   *                 decompressors this parameter has no impact on the function of the factory
+   */
+  public CodecFactory(Configuration configuration, int pageSize) {
+    this.configuration = configuration;
+    this.pageSize = pageSize;
+  }
+
+  /**
+   * Create a codec factory that will provide compressors and decompressors
+   * that will work natively with ByteBuffers backed by direct memory.
+   *
+   * @param config configuration options for different compression codecs
+   * @param allocator an allocator for creating result buffers during compression
+   *                  and decompression, must provide buffers backed by Direct
+   *                  memory and return true for the isDirect() method
+   *                  on the ByteBufferAllocator interface
+   * @param pageSize the default page size. This does not set a hard limit on the
+   *                 size of buffers that can be compressed, but performance may
+   *                 be improved by setting it close to the expected size of buffers
+   *                 (in the case of parquet, pages) that will be compressed. This
+   *                 setting is unused in the case of decompressing data, as parquet
+   *                 always records the uncompressed size of a buffer. If this
+   *                 CodecFactory is only going to be used for decompressors, this
+   *                 parameter will not impact the function of the factory.
+   */
+  public static CodecFactory createDirectCodecFactory(Configuration config, ByteBufferAllocator allocator, int pageSize) {
+    return new DirectCodecFactory(config, allocator, pageSize);
+  }
+
+  class HeapBytesDecompressor extends BytesDecompressor {
 
     private final CompressionCodec codec;
     private final Decompressor decompressor;
 
-    public BytesDecompressor(CompressionCodec codec) {
-      this.codec = codec;
+    HeapBytesDecompressor(CompressionCodecName codecName) {
+      this.codec = getCodec(codecName);
       if (codec != null) {
         decompressor = CodecPool.getDecompressor(codec);
       } else {
@@ -53,11 +99,12 @@ class CodecFactory {
       }
     }
 
+    @Override
     public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException {
       final BytesInput decompressed;
       if (codec != null) {
         decompressor.reset();
-        InputStream is = codec.createInputStream(new ByteArrayInputStream(bytes.toByteArray()), decompressor);
+        InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
         decompressed = BytesInput.from(is, uncompressedSize);
       } else {
         decompressed = bytes;
@@ -65,7 +112,13 @@ class CodecFactory {
       return decompressed;
     }
 
-    private void release() {
+    @Override
+    public void decompress(ByteBuffer input, int compressedSize, ByteBuffer output, int uncompressedSize) throws IOException {
+      ByteBuffer decompressed = decompress(BytesInput.from(input, 0, input.remaining()), uncompressedSize).toByteBuffer();
+      output.put(decompressed);
+    }
+
+    protected void release() {
       if (decompressor != null) {
         CodecPool.returnDecompressor(decompressor);
       }
@@ -78,16 +131,16 @@ class CodecFactory {
    * @author Julien Le Dem
    *
    */
-  public static class BytesCompressor {
+  class HeapBytesCompressor extends BytesCompressor {
 
     private final CompressionCodec codec;
     private final Compressor compressor;
     private final ByteArrayOutputStream compressedOutBuffer;
     private final CompressionCodecName codecName;
 
-    public BytesCompressor(CompressionCodecName codecName, CompressionCodec codec, int pageSize) {
+    HeapBytesCompressor(CompressionCodecName codecName) {
       this.codecName = codecName;
-      this.codec = codec;
+      this.codec = getCodec(codecName);
       if (codec != null) {
         this.compressor = CodecPool.getCompressor(codec);
         this.compressedOutBuffer = new ByteArrayOutputStream(pageSize);
@@ -97,6 +150,7 @@ class CodecFactory {
       }
     }
 
+    @Override
     public BytesInput compress(BytesInput bytes) throws IOException {
       final BytesInput compressedBytes;
       if (codec == null) {
@@ -116,7 +170,8 @@ class CodecFactory {
       return compressedBytes;
     }
 
-    private void release() {
+    @Override
+    protected void release() {
       if (compressor != null) {
         CodecPool.returnCompressor(compressor);
       }
@@ -128,60 +183,58 @@ class CodecFactory {
 
   }
 
-  private final Map<CompressionCodecName, BytesCompressor> compressors = new HashMap<CompressionCodecName, BytesCompressor>();
-  private final Map<CompressionCodecName, BytesDecompressor> decompressors = new HashMap<CompressionCodecName, BytesDecompressor>();
-  private final Map<String, CompressionCodec> codecByName = new HashMap<String, CompressionCodec>();
-  private final Configuration configuration;
+  public BytesCompressor getCompressor(CompressionCodecName codecName) {
+    BytesCompressor comp = compressors.get(codecName);
+    if (comp == null) {
+      comp = createCompressor(codecName);
+      compressors.put(codecName, comp);
+    }
+    return comp;
+  }
 
-  public CodecFactory(Configuration configuration) {
-    this.configuration = configuration;
+  public BytesDecompressor getDecompressor(CompressionCodecName codecName) {
+    BytesDecompressor decomp = decompressors.get(codecName);
+    if (decomp == null) {
+      decomp = createDecompressor(codecName);
+      decompressors.put(codecName, decomp);
+    }
+    return decomp;
+  }
+
+  protected BytesCompressor createCompressor(CompressionCodecName codecName) {
+    return new HeapBytesCompressor(codecName);
+  }
+
+  protected BytesDecompressor createDecompressor(CompressionCodecName codecName) {
+    return new HeapBytesDecompressor(codecName);
   }
 
   /**
    *
-   * @param codecName the requested codec
+   * @param codecName
+   *          the requested codec
    * @return the corresponding hadoop codec. null if UNCOMPRESSED
    */
-  private CompressionCodec getCodec(CompressionCodecName codecName) {
+  protected CompressionCodec getCodec(CompressionCodecName codecName) {
     String codecClassName = codecName.getHadoopCompressionCodecClassName();
     if (codecClassName == null) {
       return null;
     }
-    CompressionCodec codec = codecByName.get(codecClassName);
+    CompressionCodec codec = CODEC_BY_NAME.get(codecClassName);
     if (codec != null) {
       return codec;
     }
 
     try {
       Class<?> codecClass = Class.forName(codecClassName);
-      codec = (CompressionCodec)ReflectionUtils.newInstance(codecClass, configuration);
-      codecByName.put(codecClassName, codec);
+      codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, configuration);
+      CODEC_BY_NAME.put(codecClassName, codec);
       return codec;
     } catch (ClassNotFoundException e) {
       throw new BadConfigurationException("Class " + codecClassName + " was not found", e);
     }
   }
 
-  public BytesCompressor getCompressor(CompressionCodecName codecName, int pageSize) {
-    BytesCompressor comp = compressors.get(codecName);
-    if (comp == null) {
-      CompressionCodec codec = getCodec(codecName);
-      comp = new BytesCompressor(codecName, codec, pageSize);
-      compressors.put(codecName, comp);
-    }
-    return comp;
-  }
-
-  public BytesDecompressor getDecompressor(CompressionCodecName codecName) {
-    BytesDecompressor decomp = decompressors.get(codecName);
-    if (decomp == null) {
-      CompressionCodec codec = getCodec(codecName);
-      decomp = new BytesDecompressor(codec);
-      decompressors.put(codecName, decomp);
-    }
-    return decomp;
-  }
-
   public void release() {
     for (BytesCompressor compressor : compressors.values()) {
       compressor.release();
@@ -192,4 +245,16 @@ class CodecFactory {
     }
     decompressors.clear();
   }
+
+  public static abstract class BytesCompressor {
+    public abstract BytesInput compress(BytesInput bytes) throws IOException;
+    public abstract CompressionCodecName getCodecName();
+    protected abstract void release();
+  }
+
+  public static abstract class BytesDecompressor {
+    public abstract BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException;
+    public abstract void decompress(ByteBuffer input, int compressedSize, ByteBuffer output, int uncompressedSize) throws IOException;
+    protected abstract void release();
+  }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java
index b6934c2..af06747 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *   http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY


[4/4] parquet-mr git commit: PARQUET-77: ByteBuffer use in read and write paths

Posted by ju...@apache.org.
PARQUET-77: ByteBuffer use in read and write paths

This work is based on the GSOC project from the summer of 2014. We have expanded on it to fix bugs and change the write path to use ByteBuffers as well. This PR replaces the earlier PRs #6, #49 and #50

Author: Jason Altekruse <al...@gmail.com>
Author: sunyu <st...@gmail.com>
Author: adeneche <ad...@gmail.com>
Author: Jacques Nadeau <ja...@apache.org>
Author: Parth Chandra <pc...@maprtech.com>
Author: stormdsy@gmail.com <st...@gmail.com>
Author: Jason Altekruse <al...@open-math.com>
Author: dsy <st...@gmail.com>
Author: Steven Phillips <sp...@maprtech.com>
Author: Gera Shegalov <ge...@twitter.com>
Author: Ryan Blue <bl...@apache.org>

Closes #267 from jaltekruse/1.6.0rc3-drill-r0.3-merge and squashes the following commits:

56316d0 [Jason Altekruse] An exception out of the read method doesn't necessarily mean something is very wrong, so it shouldn't get wrapped in a ShouldNeverHappenException. This invocationTargetException will wrap any kind of exception coming out of the method, including an IOException.
58340d8 [Jason Altekruse] Fix CompatibilityUtil, primary issue was a small error in the package name for the class that was being used to detect if the Hadoop 2.x API was available.
96e19a8 [Jason Altekruse] Properly set the byte buffer position when reading out of a filesystem that does not implement the byte buffer based read method in the Hadoop 2.x API.
269daef [Jason Altekruse] Make CodecFactory public
bd7aa97 [Jason Altekruse] Remove unused imports, one of which has been moved to package private and is no longer accessible in this class.
a44fdba [Jason Altekruse] Fix logging and restrict access to classes inside of CodecFactory.
723701c [Jason Altekruse] Adding isDirect interface to ByteBufferAllocator to add a restriction on the allocators used by a DirectCodecFactory.
10b5ba3 [Jason Altekruse] Remove unneeded TODO
57491a2 [Jason Altekruse] Delete older version of test file, all of these tests look to be covered in the newer version.
d6501b1 [Jason Altekruse] Thought I had fixed this double deallocation earlier, guess the change got lost somewhere.
a8d2dc1 [Jason Altekruse] Address review comments.
40714a4 [Jason Altekruse] Move pageSize to the constructor of codecfactory rather than the method for getting a compressor.
df7fd9c [Jason Altekruse] Limit access to classes and methods used for reflection based access to Hadoop 2.0 compression APIs.
192c717 [Jason Altekruse] Fix error message
1a47767 [Jason Altekruse] Address review comments
5869156 [Jason Altekruse] Move fallback classes from HeapCodecFactory to the DirectCodecFactory
3945674 [Jason Altekruse] Switch to using the DirectCodecFactory everywhere, one test is failing form the command line that is passing in intellij.
e7f7f7f [Jason Altekruse] WIP - removing unneeded generics form CodecFactories
659230f [Jason Altekruse] Remove second version of the class ByteBufferBytesInput that was nested in DirectCodecFactory. Replace with the one that was declared in the BytesInput class.
c305984 [Jason Altekruse] Adding back code generation for method to take a byte array as well as the new implementation that takes a Bytebuffer.
b8f54c2 [Jason Altekruse] Add a unit test for ByteBufferBackedBinary.
ae58486 [Jason Altekruse] Changing argument lists that previously included both an allocator and a ParquetProperties object.
b4266fb [Jason Altekruse] Add license header to new class
f8e5988 [Jason Altekruse] Added javadocs, removed unused code in DirectCodecFactory
d332ca7 [Jason Altekruse] Add test for UnsignedVarIntBytesInput
b7a6457 [Jason Altekruse] fix license leader
8ff878a [Jason Altekruse] Addressing review comments
862eb13 [Jason Altekruse] Fix usage of old constructor in Thrift module that caused a compilation failure. I had been skipping this module entirely during my work as the tests will fail to compile without a binary version of thrift 0.7, which seems hard to come by or compile yourself on Mac OS X.
0496350 [Jason Altekruse] Add unit test for direct codec factory.
da1b52a [Jason Altekruse] Moving classes into parquet from Drill.
2f1a6c7 [Jason Altekruse] Consolidate a little more code
8f66e43 [Jason Altekruse] Create utility methods to transform checked exceptions to unchecked when using reflection.
f217e6a [Jason Altekruse] Restore old interfaces
d5536b6 [Jason Altekruse] Restore original name of CapacityByteArrayOutputStream to keep compatibility with 1.7
4c3195e [Jason Altekruse] Turn back on SemVer
2e95915 [Jason Altekruse] Addressing minor review comments, comments out code, star import, formatting
a793be8 [Jason Altekruse] Add closeQuietly method to convert checked  IOExceptions from classless into runtime exceptions. Remove a bunch of unused imports from when there were previously try catch blocks that did this wrapping themselves (many actually were refactored to remove any need for special exception handling in an earlier commit, only one is actually using the new method).
fdb689c [Jason Altekruse] Remove unnecessary copy writing a Binary to an OutputStream if it is backed by a byte array.
d4819b4 [Jason Altekruse] remove methods now unneccesary as same implementation has been moved to the base class.
ad58bbe [Jason Altekruse] Addressing small review comments, unused imports, doc cleanup, etc.
9fb65dd [Jason Altekruse] Rename method to get a dictionary page to clarify that the dictionary will be closed and not available for further insertion.
e79684e [Jason Altekruse] Review comments - fixing use of ParquetProperties and removing unused interfaces on PageWriter
b1040a8 [Jason Altekruse] Remove code used to debug a test that was failing after the initial merge.
9dccb94 [Jason Altekruse] Add new method to turn BytesInput into an InputStream.
f0e31ec [Jason Altekruse] revert small formatting and renaming changes, TODO make sure these result in a net diff of no changes (or only intended functional changes)
0098b1c [Jason Altekruse] Remove unused method
8c6e4a9 [Jason Altekruse] Addressing review comments, moving code out of generated class into abstract base class.
29cc747 [Jason Altekruse] Factor out common code
6959db7 [Jason Altekruse] addressing review comments, avoiding unnecessary copies when creating ByteBuffers
fec4242 [Jason Altekruse] Address review comments - factoring out code in tests
104a1d1 [Jason Altekruse] Remove test requiring a hard-coded binary file. This was actually a bad file being produced by Drill because we were not flushing the RecordConsumer.
86317b0 [Jason Altekruse] Address review comments, make field in immutable ParquetProperties object final, make an interface now expecting a ByteBuffer deprecated for the version that takes a byte[].
1971fc5 [Jason Altekruse] Fixes made while debugging drill unit tests
ebae775 [Jason Altekruse] Fix issue reading page data into an off-heap ByteBuffer
705b864 [Jason Altekruse] Rename CapacityByteArrayOutputStream to CapacityByteBufferOutputStream to reflect new implementation internals. Add close method to CapacityByteBufferOutputStream and a few other classes.
35d8386 [Jason Altekruse] Move call to getBytes() on dictionaryPages to remove the need to cache a list of dictionaryEncoders to be closed later.
d40706b [Jason Altekruse] Get rid of unnecessary calls to Bytebuffer.wrap(byte[]), as an interface that takes a byte array is still available.
fddd4af [Jason Altekruse] WIP - removing copies from the ByteBufferBasedBinary equals, compareTo, hashCode methods. Current tests are passing, but I should add some new ones.
829af6f [Jason Altekruse] WIP - getting rid of unnecessary copies in Binary.java
23ad48e [Jason Altekruse] WIP - addressing review comments
7e252f3 [Jason Altekruse] WIP - addressing review comments
1f4f504 [Jason Altekruse] WIP - addressing review comments
ab54c4e [Jason Altekruse] Moving classes out of the old packages.
45cadee [Jason Altekruse] Cleaning up code in Binary after merge.
864b011 [Jason Altekruse] Simplifying how buffer allocators are passed when creating ValuesWriters.
2b8328b [Jason Altekruse] I all of the tests are now passing after the merge.
1bfa3a0 [Jason Altekruse] Merge branch 'master' into 1.6.0rc3-drill-r0.3-merge
9bbc269 [Jacques Nadeau] Update to 1.6.0rc3-drill-r0.3
9f22bd7 [Jacques Nadeau] Make CodecFactory pluggable
4a9dd28 [Jacques Nadeau] update pom version
173aa25 [Jacques Nadeau] Set max preferred slab size to 16mb
c98ec2a [adeneche] bumped version to 1.6.0rc3-drill-r0.1
51cf2f1 [Ryan Blue] cherry pick pull#188
e1df3b9 [adeneche] disabled enforcer and changed version to -drill
6943536 [adeneche] fixing bug related to testDictionaryError_419
48cceef [Steven Phillips] Fix allocation in DictionaryValuesWriter
98b99ea [Parth Chandra] Revert readFooter to not use ZeroCopy path.
a6389db [Steven Phillips] Make constructor for PrimitiveType that takes decimalMetadata public.
e488924 [adeneche] after merge code cleanup
35b10af [Parth Chandra] Use ByteBuffers in the Write path. Allow callers to pass in an allocator to allocate the ByteBuffer.
2187697 [Jacques Nadeau] Update Binary to make a copy of data for initial statistics.
8143174 [adeneche] update pig.version to build with Hadoop 2 jars
2c2b183 [Parth Chandra] Remove Zero Copy read path while reading footers
7bc2a4d [Parth Chandra] Make a copy of Min and Max values for BinaryStatistics so that direct memory can be released before stats are written.
5bc8774 [Parth Chandra] Update Snappy Codec to implement DirectDecompressionCodec interface Add compatibility function to read directly into a byte buffer
0d22908 [adeneche] merging with master
8be638a [sunyu] Address tsdeng's comments
861e541 [dsy] enable enforcer check.
912cbaf [sunyu] fix a bug in equals in ByteBuffer Binary with offset and length
016e89c [sunyu] remove some unncessary codes. add compatible method initFromPage in ValueReaders. add toByteBuffer method in ByteBufferInputStream. add V21FileAPI class to encapsulate v21 APIs and make it a singlton. add ByteBuffer based equal and compareto method in Binary.
26dc879 [dsy] disable enforcer to pass build.
a7bcfbb [sunyu] Make BytePacker consume ByteBuffer directly.
01c2ae5 [sunyu] Implement FSDISTransport in Compatible layer. Fix bugs in Binary.
47b177d [sunyu] Move CompatibilityUtil to parquet.hadoop.util. Use reflect to call new API to keep compatible.
970fc8b [stormdsy@gmail.com] Add a Hadoop compatible layer to abstract away the zero copy API and old API.
4f399aa [stormdsy@gmail.com] Add original readIntLittleEndian function to keep compatible with previous verision.
7ac1df5 [stormdsy@gmail.com] Using Writable Channel to replace write to OutputStream one by one.
36aba13 [sunyu] Read from ByteBuffer instead of ByteArray to avoid unnecessary array copy through read path.
53500d4 [sunyu] Add ByteBufferInputStream and modify Chunk to consume ByteBuffer instead of byte array.
df1ad93 [stormdsy@gmail.com] Reading chunk using zero-copy API
2d32f49 [Gera Shegalov] Reading file metadata using zero-copy API
686d598 [Gera Shegalov] Use ByteBuf-based api to read magic.


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

Branch: refs/heads/master
Commit: 6b605a4ea05b66e1a6bf843353abcb4834a4ced8
Parents: 5a45ae3
Author: Jason Altekruse <al...@gmail.com>
Authored: Wed Nov 4 09:13:09 2015 -0800
Committer: Julien Le Dem <ju...@ledem.net>
Committed: Wed Nov 4 09:13:09 2015 -0800

----------------------------------------------------------------------
 .../apache/parquet/column/ColumnWriteStore.java |   6 +
 .../org/apache/parquet/column/ColumnWriter.java |  11 +
 .../parquet/column/ParquetProperties.java       |  53 +-
 .../parquet/column/impl/ColumnReaderImpl.java   |  11 +-
 .../parquet/column/impl/ColumnWriteStoreV1.java |  14 +-
 .../parquet/column/impl/ColumnWriteStoreV2.java |  11 +
 .../parquet/column/impl/ColumnWriterV1.java     |  22 +-
 .../parquet/column/impl/ColumnWriterV2.java     |  27 +-
 .../apache/parquet/column/page/PageWriter.java  |   2 +-
 .../parquet/column/values/ValuesReader.java     |  19 +-
 .../parquet/column/values/ValuesWriter.java     |  13 +-
 .../bitpacking/BitPackingValuesReader.java      |  13 +-
 .../bitpacking/BitPackingValuesWriter.java      |  10 +-
 .../bitpacking/ByteBitPackingValuesReader.java  |  14 +-
 .../bitpacking/ByteBitPackingValuesWriter.java  |   1 +
 .../column/values/boundedint/BitReader.java     |   7 +-
 .../column/values/boundedint/BitWriter.java     |  12 +-
 .../boundedint/BoundedIntValuesFactory.java     |   5 +-
 .../boundedint/BoundedIntValuesReader.java      |   5 +-
 .../boundedint/BoundedIntValuesWriter.java      |  10 +-
 .../boundedint/ZeroIntegerValuesReader.java     |   3 +-
 .../delta/DeltaBinaryPackingValuesReader.java   |  15 +-
 .../delta/DeltaBinaryPackingValuesWriter.java   |  17 +-
 .../DeltaLengthByteArrayValuesReader.java       |  11 +-
 .../DeltaLengthByteArrayValuesWriter.java       |  14 +-
 .../deltastrings/DeltaByteArrayReader.java      |   5 +-
 .../deltastrings/DeltaByteArrayWriter.java      |  14 +-
 .../dictionary/DictionaryValuesReader.java      |  13 +-
 .../dictionary/DictionaryValuesWriter.java      |  88 ++--
 .../dictionary/PlainValuesDictionary.java       |  26 +-
 .../values/fallback/FallbackValuesWriter.java   |  12 +-
 .../values/plain/BinaryPlainValuesReader.java   |  10 +-
 .../values/plain/BooleanPlainValuesReader.java  |   5 +-
 .../values/plain/BooleanPlainValuesWriter.java  |   6 +
 .../FixedLenByteArrayPlainValuesReader.java     |   9 +-
 .../FixedLenByteArrayPlainValuesWriter.java     |  13 +-
 .../column/values/plain/PlainValuesReader.java  |  15 +-
 .../column/values/plain/PlainValuesWriter.java  |  11 +-
 .../rle/RunLengthBitPackingHybridDecoder.java   |   8 +-
 .../rle/RunLengthBitPackingHybridEncoder.java   |  10 +-
 .../RunLengthBitPackingHybridValuesReader.java  |   7 +-
 .../RunLengthBitPackingHybridValuesWriter.java  |  10 +-
 .../java/org/apache/parquet/io/api/Binary.java  | 204 ++++++--
 .../apache/parquet/schema/PrimitiveType.java    |   7 +-
 .../column/impl/TestColumnReaderImpl.java       |   1 +
 .../column/impl/TestCorruptDeltaByteArrays.java |  31 +-
 .../parquet/column/mem/TestMemColumn.java       |   3 +-
 .../parquet/column/page/mem/MemPageWriter.java  |   1 -
 .../org/apache/parquet/column/values/Utils.java |   5 +-
 .../values/bitpacking/BitPackingPerfTest.java   |   3 +-
 .../values/bitpacking/TestBitPackingColumn.java |   6 +-
 .../values/boundedint/TestBoundedColumns.java   |  11 +-
 .../DeltaBinaryPackingValuesWriterTest.java     |  12 +-
 .../benchmark/BenchmarkIntegerOutputSize.java   |   5 +-
 .../BenchmarkReadingRandomIntegers.java         |   8 +-
 .../benchmark/RandomWritingBenchmarkTest.java   |   7 +-
 .../SmallRangeWritingBenchmarkTest.java         |   3 +-
 .../TestDeltaLengthByteArray.java               |  13 +-
 .../BenchmarkDeltaLengthByteArray.java          |   5 +-
 .../values/deltastrings/TestDeltaByteArray.java |   9 +-
 .../benchmark/BenchmarkDeltaByteArray.java      |   9 +-
 .../values/dictionary/TestDictionary.java       |  50 +-
 ...unLengthBitPackingHybridIntegrationTest.java |  10 +-
 .../TestRunLengthBitPackingHybridEncoder.java   |  30 +-
 .../java/org/apache/parquet/io/PerfTest.java    |   3 +-
 .../org/apache/parquet/io/TestColumnIO.java     |  24 +-
 .../org/apache/parquet/io/TestFiltered.java     |   3 +-
 .../org/apache/parquet/io/api/TestBinary.java   |  25 +
 .../org/apache/parquet/IOExceptionUtils.java    |  43 ++
 .../parquet/OutputStreamCloseException.java     |  46 ++
 .../apache/parquet/ParquetRuntimeException.java |   4 +-
 .../parquet/bytes/ByteBufferAllocator.java      |  38 ++
 .../parquet/bytes/ByteBufferInputStream.java    |  82 +++
 .../org/apache/parquet/bytes/BytesUtils.java    |  24 +
 .../bytes/DirectByteBufferAllocator.java        |  43 ++
 .../parquet/bytes/HeapByteBufferAllocator.java  |  44 ++
 .../org/apache/parquet/bytes/BytesInput.java    |  84 ++-
 .../bytes/CapacityByteArrayOutputStream.java    | 107 +++-
 .../bytes/LittleEndianDataOutputStream.java     |   7 +
 .../column/values/bitpacking/BytePacker.java    |  21 +-
 .../apache/parquet/bytes/TestBytesInput.java    |  42 ++
 .../TestCapacityByteArrayOutputStream.java      |   6 +-
 .../values/bitpacking/TestByteBitPacking.java   |   5 +-
 .../values/bitpacking/TestLemireBitPacking.java |   3 +-
 .../ByteBasedBitPackingGenerator.java           |  26 +-
 parquet-hadoop/pom.xml                          |   5 +
 .../converter/ParquetMetadataConverter.java     |   1 +
 .../org/apache/parquet/hadoop/CodecFactory.java | 157 ++++--
 .../hadoop/ColumnChunkPageReadStore.java        |   6 +-
 .../hadoop/ColumnChunkPageWriteStore.java       |  34 +-
 .../parquet/hadoop/DirectCodecFactory.java      | 522 +++++++++++++++++++
 .../hadoop/InternalParquetRecordWriter.java     |  21 +-
 .../parquet/hadoop/ParquetFileReader.java       |  81 +--
 .../parquet/hadoop/ParquetFileWriter.java       |   3 +-
 .../parquet/hadoop/ParquetOutputFormat.java     |   5 +-
 .../parquet/hadoop/ParquetRecordWriter.java     |  14 +-
 .../apache/parquet/hadoop/ParquetWriter.java    |   8 +-
 .../hadoop/codec/SnappyDecompressor.java        |   3 +-
 .../parquet/hadoop/util/CompatibilityUtil.java  | 114 ++++
 .../hadoop/TestColumnChunkPageWriteStore.java   |   9 +-
 .../parquet/hadoop/TestDirectCodecFactory.java  | 165 ++++++
 .../parquet/pig/TupleConsumerPerfTest.java      |   3 +-
 .../parquet/thrift/TestParquetReadProtocol.java |   4 +-
 pom.xml                                         |   3 +-
 104 files changed, 2396 insertions(+), 463 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriteStore.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriteStore.java b/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriteStore.java
index 739c00f..bb9dfea 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriteStore.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriteStore.java
@@ -58,4 +58,10 @@ public interface ColumnWriteStore {
    * @return a formated string representing memory usage per column
    */
   abstract public String memUsageString();
+
+  /**
+   * Close the related output stream and release any resources
+   */
+  abstract public void close();
+
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriter.java
index 7605c50..c824504 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/ColumnWriter.java
@@ -83,5 +83,16 @@ public interface ColumnWriter {
    */
   void writeNull(int repetitionLevel, int definitionLevel);
 
+ /**
+  * Close the underlying store. This should be called when there are no
+  * more data to be written.
+  */
+  void close();
+
+  /**
+   * used to decide when to write a page or row group
+   * @return the number of bytes of memory used to buffer the current data
+   */
+  long getBufferedSizeInMemory();
 }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
index df44c4b..f8567a8 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
@@ -18,6 +18,10 @@
  */
 package org.apache.parquet.column;
 
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferAllocator;
+import org.apache.parquet.bytes.HeapByteBufferAllocator;
+
 import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt;
 import static org.apache.parquet.column.Encoding.PLAIN;
 import static org.apache.parquet.column.Encoding.PLAIN_DICTIONARY;
@@ -74,19 +78,27 @@ public class ParquetProperties {
   private final int dictionaryPageSizeThreshold;
   private final WriterVersion writerVersion;
   private final boolean enableDictionary;
+  private final ByteBufferAllocator allocator;
 
   public ParquetProperties(int dictPageSize, WriterVersion writerVersion, boolean enableDict) {
+    this(dictPageSize, writerVersion, enableDict, new HeapByteBufferAllocator());
+  }
+
+  public ParquetProperties(int dictPageSize, WriterVersion writerVersion, boolean enableDict, ByteBufferAllocator allocator) {
     this.dictionaryPageSizeThreshold = dictPageSize;
     this.writerVersion = writerVersion;
     this.enableDictionary = enableDict;
+    Preconditions.checkNotNull(allocator, "ByteBufferAllocator");
+    this.allocator = allocator;
   }
 
-  public static ValuesWriter getColumnDescriptorValuesWriter(int maxLevel, int initialSizePerCol, int pageSize) {
+  public ValuesWriter getColumnDescriptorValuesWriter(int maxLevel, int initialSizePerCol, int pageSize) {
     if (maxLevel == 0) {
       return new DevNullValuesWriter();
     } else {
       return new RunLengthBitPackingHybridValuesWriter(
-          getWidthFromMaxInt(maxLevel), initialSizePerCol, pageSize);
+          getWidthFromMaxInt(maxLevel), initialSizePerCol, pageSize, this.allocator
+      );
     }
   }
 
@@ -95,15 +107,15 @@ public class ParquetProperties {
     case BOOLEAN:
       return new BooleanPlainValuesWriter();
     case INT96:
-      return new FixedLenByteArrayPlainValuesWriter(12, initialSizePerCol, pageSize);
+      return new FixedLenByteArrayPlainValuesWriter(12, initialSizePerCol, pageSize, this.allocator);
     case FIXED_LEN_BYTE_ARRAY:
-      return new FixedLenByteArrayPlainValuesWriter(path.getTypeLength(), initialSizePerCol, pageSize);
+      return new FixedLenByteArrayPlainValuesWriter(path.getTypeLength(), initialSizePerCol, pageSize, this.allocator);
     case BINARY:
     case INT32:
     case INT64:
     case DOUBLE:
     case FLOAT:
-      return new PlainValuesWriter(initialSizePerCol, pageSize);
+      return new PlainValuesWriter(initialSizePerCol, pageSize, this.allocator);
     default:
       throw new IllegalArgumentException("Unknown type " + path.getType());
     }
@@ -128,19 +140,19 @@ public class ParquetProperties {
     case BOOLEAN:
       throw new IllegalArgumentException("no dictionary encoding for BOOLEAN");
     case BINARY:
-      return new PlainBinaryDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage);
+      return new PlainBinaryDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage, this.allocator);
     case INT32:
-      return new PlainIntegerDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage);
+      return new PlainIntegerDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage, this.allocator);
     case INT64:
-      return new PlainLongDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage);
+      return new PlainLongDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage, this.allocator);
     case INT96:
-      return new PlainFixedLenArrayDictionaryValuesWriter(dictionaryPageSizeThreshold, 12, encodingForDataPage, encodingForDictionaryPage);
+      return new PlainFixedLenArrayDictionaryValuesWriter(dictionaryPageSizeThreshold, 12, encodingForDataPage, encodingForDictionaryPage, this.allocator);
     case DOUBLE:
-      return new PlainDoubleDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage);
+      return new PlainDoubleDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage, this.allocator);
     case FLOAT:
-      return new PlainFloatDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage);
+      return new PlainFloatDictionaryValuesWriter(dictionaryPageSizeThreshold, encodingForDataPage, encodingForDictionaryPage, this.allocator);
     case FIXED_LEN_BYTE_ARRAY:
-      return new PlainFixedLenArrayDictionaryValuesWriter(dictionaryPageSizeThreshold, path.getTypeLength(), encodingForDataPage, encodingForDictionaryPage);
+      return new PlainFixedLenArrayDictionaryValuesWriter(dictionaryPageSizeThreshold, path.getTypeLength(), encodingForDataPage, encodingForDictionaryPage, this.allocator);
     default:
       throw new IllegalArgumentException("Unknown type " + path.getType());
     }
@@ -153,12 +165,12 @@ public class ParquetProperties {
     case PARQUET_2_0:
       switch (path.getType()) {
       case BOOLEAN:
-        return new RunLengthBitPackingHybridValuesWriter(1, initialSizePerCol, pageSize);
+        return new RunLengthBitPackingHybridValuesWriter(1, initialSizePerCol, pageSize, this.allocator);
       case BINARY:
       case FIXED_LEN_BYTE_ARRAY:
-        return new DeltaByteArrayWriter(initialSizePerCol, pageSize);
+        return new DeltaByteArrayWriter(initialSizePerCol, pageSize,this.allocator);
       case INT32:
-        return new DeltaBinaryPackingValuesWriter(initialSizePerCol, pageSize);
+        return new DeltaBinaryPackingValuesWriter(initialSizePerCol, pageSize, this.allocator);
       case INT96:
       case INT64:
       case DOUBLE:
@@ -218,23 +230,28 @@ public class ParquetProperties {
     return enableDictionary;
   }
 
+  public ByteBufferAllocator getAllocator() {
+    return allocator;
+  }
+
   public ColumnWriteStore newColumnWriteStore(
       MessageType schema,
       PageWriteStore pageStore,
-      int pageSize) {
+      int pageSize,
+      ByteBufferAllocator allocator) {
     switch (writerVersion) {
     case PARQUET_1_0:
       return new ColumnWriteStoreV1(
           pageStore,
           pageSize,
           dictionaryPageSizeThreshold,
-          enableDictionary, writerVersion);
+          enableDictionary, writerVersion, allocator);
     case PARQUET_2_0:
       return new ColumnWriteStoreV2(
           schema,
           pageStore,
           pageSize,
-          new ParquetProperties(dictionaryPageSizeThreshold, writerVersion, enableDictionary));
+          new ParquetProperties(dictionaryPageSizeThreshold, writerVersion, enableDictionary, allocator));
     default:
       throw new IllegalArgumentException("unknown version " + writerVersion);
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
index c53977f..8c2a4bf 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
@@ -27,6 +27,7 @@ import static org.apache.parquet.column.ValuesType.VALUES;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.CorruptDeltaByteArrays;
 import org.apache.parquet.Log;
@@ -548,7 +549,7 @@ public class ColumnReaderImpl implements ColumnReader {
     });
   }
 
-  private void initDataReader(Encoding dataEncoding, byte[] bytes, int offset, int valueCount) {
+  private void initDataReader(Encoding dataEncoding, ByteBuffer bytes, int offset, int valueCount) {
     ValuesReader previousReader = this.dataColumn;
 
     this.currentEncoding = dataEncoding;
@@ -588,8 +589,8 @@ public class ColumnReaderImpl implements ColumnReader {
     this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader);
     this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader);
     try {
-      byte[] bytes = page.getBytes().toByteArray();
-      if (DEBUG) LOG.debug("page size " + bytes.length + " bytes and " + pageValueCount + " records");
+      ByteBuffer bytes = page.getBytes().toByteBuffer();
+      if (DEBUG) LOG.debug("page size " + bytes.remaining() + " bytes and " + pageValueCount + " records");
       if (DEBUG) LOG.debug("reading repetition levels at 0");
       rlReader.initFromPage(pageValueCount, bytes, 0);
       int next = rlReader.getNextOffset();
@@ -608,7 +609,7 @@ public class ColumnReaderImpl implements ColumnReader {
     this.definitionLevelColumn = newRLEIterator(path.getMaxDefinitionLevel(), page.getDefinitionLevels());
     try {
       if (DEBUG) LOG.debug("page data size " + page.getData().size() + " bytes and " + pageValueCount + " records");
-      initDataReader(page.getDataEncoding(), page.getData().toByteArray(), 0, page.getValueCount());
+      initDataReader(page.getDataEncoding(), page.getData().toByteBuffer(), 0, page.getValueCount());
     } catch (IOException e) {
       throw new ParquetDecodingException("could not read page " + page + " in col " + path, e);
     }
@@ -622,7 +623,7 @@ public class ColumnReaderImpl implements ColumnReader {
       return new RLEIntIterator(
           new RunLengthBitPackingHybridDecoder(
               BytesUtils.getWidthFromMaxInt(maxLevel),
-              new ByteArrayInputStream(bytes.toByteArray())));
+              bytes.toInputStream()));
     } catch (IOException e) {
       throw new ParquetDecodingException("could not read levels in page for col " + path, e);
     }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java
index a72b6f7..277c468 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java
@@ -25,6 +25,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.ColumnWriteStore;
 import org.apache.parquet.column.ColumnWriter;
@@ -40,14 +41,16 @@ public class ColumnWriteStoreV1 implements ColumnWriteStore {
   private final int dictionaryPageSizeThreshold;
   private final boolean enableDictionary;
   private final WriterVersion writerVersion;
+  private final ByteBufferAllocator allocator;
 
-  public ColumnWriteStoreV1(PageWriteStore pageWriteStore, int pageSizeThreshold, int dictionaryPageSizeThreshold, boolean enableDictionary, WriterVersion writerVersion) {
+  public ColumnWriteStoreV1(PageWriteStore pageWriteStore, int pageSizeThreshold, int dictionaryPageSizeThreshold, boolean enableDictionary, WriterVersion writerVersion, ByteBufferAllocator allocator) {
     super();
     this.pageWriteStore = pageWriteStore;
     this.pageSizeThreshold = pageSizeThreshold;
     this.dictionaryPageSizeThreshold = dictionaryPageSizeThreshold;
     this.enableDictionary = enableDictionary;
     this.writerVersion = writerVersion;
+    this.allocator = allocator;
   }
 
   public ColumnWriter getColumnWriter(ColumnDescriptor path) {
@@ -65,7 +68,7 @@ public class ColumnWriteStoreV1 implements ColumnWriteStore {
 
   private ColumnWriterV1 newMemColumn(ColumnDescriptor path) {
     PageWriter pageWriter = pageWriteStore.getPageWriter(path);
-    return new ColumnWriterV1(path, pageWriter, pageSizeThreshold, dictionaryPageSizeThreshold, enableDictionary, writerVersion);
+    return new ColumnWriterV1(path, pageWriter, pageSizeThreshold, dictionaryPageSizeThreshold, enableDictionary, writerVersion, allocator);
   }
 
   @Override
@@ -132,4 +135,11 @@ public class ColumnWriteStoreV1 implements ColumnWriteStore {
     // V1 does not take record boundaries into account
   }
 
+  public void close() {
+    Collection<ColumnWriterV1> values = columns.values();
+    for (ColumnWriterV1 memColumn : values) {
+      memColumn.close();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
index fc17a22..4126004 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
@@ -29,6 +29,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.ColumnWriteStore;
 import org.apache.parquet.column.ColumnWriter;
@@ -50,6 +51,7 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
   private long rowCount;
   private long rowCountForNextSizeCheck = MINIMUM_RECORD_COUNT_FOR_CHECK;
   private final long thresholdTolerance;
+  private final ByteBufferAllocator allocator;
 
   private int pageSizeThreshold;
 
@@ -61,6 +63,7 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
     super();
     this.pageSizeThreshold = pageSizeThreshold;
     this.thresholdTolerance = (long)(pageSizeThreshold * THRESHOLD_TOLERANCE_RATIO);
+    this.allocator = parquetProps.getAllocator();
     Map<ColumnDescriptor, ColumnWriterV2> mcolumns = new TreeMap<ColumnDescriptor, ColumnWriterV2>();
     for (ColumnDescriptor path : schema.getColumns()) {
       PageWriter pageWriter = pageWriteStore.getPageWriter(path);
@@ -128,6 +131,14 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
   }
 
   @Override
+  public void close() {
+    flush(); // calling flush() here to keep it consistent with the behavior before merging with master
+    for (ColumnWriterV2 memColumn : columns.values()) {
+      memColumn.close();
+    }
+  }
+
+  @Override
   public void endRecord() {
     ++ rowCount;
     if (rowCount >= rowCountForNextSizeCheck) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
index f4079c7..f010df8 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
@@ -23,6 +23,7 @@ import static org.apache.parquet.bytes.BytesInput.concat;
 import java.io.IOException;
 
 import org.apache.parquet.Log;
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.ColumnWriter;
@@ -66,7 +67,8 @@ final class ColumnWriterV1 implements ColumnWriter {
       int pageSizeThreshold,
       int dictionaryPageSizeThreshold,
       boolean enableDictionary,
-      WriterVersion writerVersion) {
+      WriterVersion writerVersion,
+      ByteBufferAllocator allocator) {
     this.path = path;
     this.pageWriter = pageWriter;
     this.pageSizeThreshold = pageSizeThreshold;
@@ -74,10 +76,10 @@ final class ColumnWriterV1 implements ColumnWriter {
     this.valueCountForNextSizeCheck = INITIAL_COUNT_FOR_SIZE_CHECK;
     resetStatistics();
 
-    ParquetProperties parquetProps = new ParquetProperties(dictionaryPageSizeThreshold, writerVersion, enableDictionary);
+    ParquetProperties parquetProps = new ParquetProperties(dictionaryPageSizeThreshold, writerVersion, enableDictionary, allocator);
 
-    this.repetitionLevelColumn = ParquetProperties.getColumnDescriptorValuesWriter(path.getMaxRepetitionLevel(), MIN_SLAB_SIZE, pageSizeThreshold);
-    this.definitionLevelColumn = ParquetProperties.getColumnDescriptorValuesWriter(path.getMaxDefinitionLevel(), MIN_SLAB_SIZE, pageSizeThreshold);
+    this.repetitionLevelColumn = parquetProps.getColumnDescriptorValuesWriter(path.getMaxRepetitionLevel(), MIN_SLAB_SIZE, pageSizeThreshold);
+    this.definitionLevelColumn = parquetProps.getColumnDescriptorValuesWriter(path.getMaxDefinitionLevel(), MIN_SLAB_SIZE, pageSizeThreshold);
 
     int initialSlabSize = CapacityByteArrayOutputStream.initialSlabSizeHeuristic(MIN_SLAB_SIZE, pageSizeThreshold, 10);
     this.dataColumn = parquetProps.getValuesWriter(path, initialSlabSize, pageSizeThreshold);
@@ -239,7 +241,7 @@ final class ColumnWriterV1 implements ColumnWriter {
     if (valueCount > 0) {
       writePage();
     }
-    final DictionaryPage dictionaryPage = dataColumn.createDictionaryPage();
+    final DictionaryPage dictionaryPage = dataColumn.toDictPageAndClose();
     if (dictionaryPage != null) {
       if (DEBUG) LOG.debug("write dictionary");
       try {
@@ -251,6 +253,16 @@ final class ColumnWriterV1 implements ColumnWriter {
     }
   }
 
+  @Override
+  public void close() {
+    flush();
+    // Close the Values writers.
+    repetitionLevelColumn.close();
+    definitionLevelColumn.close();
+    dataColumn.close();
+  }
+
+  @Override
   public long getBufferedSizeInMemory() {
     return repetitionLevelColumn.getBufferedSize()
         + definitionLevelColumn.getBufferedSize()

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
index 5e936a2..8249b72 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 
 import org.apache.parquet.Ints;
 import org.apache.parquet.Log;
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
 import org.apache.parquet.column.ColumnDescriptor;
@@ -69,8 +70,10 @@ final class ColumnWriterV2 implements ColumnWriter {
     this.pageWriter = pageWriter;
     resetStatistics();
 
-    this.repetitionLevelColumn = new RunLengthBitPackingHybridEncoder(getWidthFromMaxInt(path.getMaxRepetitionLevel()), MIN_SLAB_SIZE, pageSize);
-    this.definitionLevelColumn = new RunLengthBitPackingHybridEncoder(getWidthFromMaxInt(path.getMaxDefinitionLevel()), MIN_SLAB_SIZE, pageSize);
+    this.repetitionLevelColumn = new RunLengthBitPackingHybridEncoder(
+        getWidthFromMaxInt(path.getMaxRepetitionLevel()), MIN_SLAB_SIZE, pageSize, parquetProps.getAllocator());
+    this.definitionLevelColumn = new RunLengthBitPackingHybridEncoder(
+        getWidthFromMaxInt(path.getMaxDefinitionLevel()), MIN_SLAB_SIZE, pageSize, parquetProps.getAllocator());
 
     int initialSlabSize = CapacityByteArrayOutputStream.initialSlabSizeHeuristic(MIN_SLAB_SIZE, pageSize, 10);
     this.dataColumn = parquetProps.getValuesWriter(path, initialSlabSize, pageSize);
@@ -113,6 +116,22 @@ final class ColumnWriterV2 implements ColumnWriter {
     ++ valueCount;
   }
 
+  @Override
+  public void close() {
+    // Close the Values writers.
+    repetitionLevelColumn.close();
+    definitionLevelColumn.close();
+    dataColumn.close();
+  }
+
+  @Override
+  public long getBufferedSizeInMemory() {
+    return repetitionLevelColumn.getBufferedSize()
+      + definitionLevelColumn.getBufferedSize()
+      + dataColumn.getBufferedSize()
+      + pageWriter.getMemSize();
+  }
+
   /**
    * writes the current value
    * @param value
@@ -208,7 +227,7 @@ final class ColumnWriterV2 implements ColumnWriter {
    * Is called right after writePage
    */
   public void finalizeColumnChunk() {
-    final DictionaryPage dictionaryPage = dataColumn.createDictionaryPage();
+    final DictionaryPage dictionaryPage = dataColumn.toDictPageAndClose();
     if (dictionaryPage != null) {
       if (DEBUG) LOG.debug("write dictionary");
       try {
@@ -252,7 +271,7 @@ final class ColumnWriterV2 implements ColumnWriter {
   }
 
   /**
-   * @param prefix a prefix to format lines
+   * @param indent a prefix to format lines
    * @return a formatted string showing how memory is used
    */
   public String memUsageString(String indent) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java
index 4ad7d9f..a1d8647 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.page;
 
 import java.io.IOException;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.statistics.Statistics;
@@ -54,7 +55,6 @@ public interface PageWriter {
    * @param dataEncoding the encoding for the data
    * @param data the data encoded with dataEncoding
    * @param statistics optional stats for this page
-   * @param metadata optional free form key values
    * @throws IOException
    */
   void writePageV2(

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java
index a3d8920..03aa2f8 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java
@@ -20,13 +20,14 @@ package org.apache.parquet.column.values;
 
 import java.io.IOException;
 
+import java.nio.ByteBuffer;
 import org.apache.parquet.io.ParquetDecodingException;
 import org.apache.parquet.io.api.Binary;
 
 /**
  * Base class to implement an encoding for a given column type.
  *
- * A ValuesReader is provided with a page (byte-array) and is responsible
+ * A ValuesReader is provided with a page (byte-buffer) and is responsible
  * for deserializing the primitive values stored in that page.
  *
  * Given that pages are homogeneous (store only a single type), typical subclasses
@@ -58,8 +59,20 @@ public abstract class ValuesReader {
    *
    * @throws IOException
    */
-  public abstract void initFromPage(int valueCount, byte[] page, int offset) throws IOException;
-  
+  public abstract void initFromPage(int valueCount, ByteBuffer page, int offset) throws IOException;
+
+  /**
+   * Same functionality as method of the same name that takes a ByteBuffer instead of a byte[].
+   *
+   * This method is only provided for backward compatibility and will be removed in a future release.
+   * Please update any code using it as soon as possible.
+   * @see #initFromPage(int, ByteBuffer, int)
+   */
+  @Deprecated
+  public void initFromPage(int valueCount, byte[] page, int offset) throws IOException {
+    this.initFromPage(valueCount, ByteBuffer.wrap(page), offset);
+  }
+
   /**
    * Called to return offset of the next section
    * @return offset of the next section

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesWriter.java
index c8f31b9..e5cf8e9 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesWriter.java
@@ -56,9 +56,20 @@ public abstract class ValuesWriter {
   public abstract void reset();
 
   /**
+   * Called to close the values writer. Any output stream is closed and can no longer be used.
+   * All resources are released.
+   */
+  public void close() {
+  }
+
+  /**
+   * Returns the dictionary generated by this writer if one was created.
+   * As part of this operation the dictionary is closed and will not have
+   * any new values written into it.
+   *
    * @return the dictionary page or null if not dictionary based
    */
-  public DictionaryPage createDictionaryPage() {
+  public DictionaryPage toDictPageAndClose() {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java
index f713263..f540c39 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesReader.java
@@ -21,9 +21,10 @@ package org.apache.parquet.column.values.bitpacking;
 import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt;
 import static org.apache.parquet.column.values.bitpacking.BitPacking.createBitPackingReader;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.values.ValuesReader;
@@ -39,7 +40,7 @@ import org.apache.parquet.io.ParquetDecodingException;
 public class BitPackingValuesReader extends ValuesReader {
   private static final Log LOG = Log.getLog(BitPackingValuesReader.class);
 
-  private ByteArrayInputStream in;
+  private ByteBufferInputStream in;
   private BitPackingReader bitPackingReader;
   private final int bitsPerValue;
   private int nextOffset;
@@ -66,18 +67,18 @@ public class BitPackingValuesReader extends ValuesReader {
 
   /**
    * {@inheritDoc}
-   * @see org.apache.parquet.column.values.ValuesReader#initFromPage(long, byte[], int)
+   * @see org.apache.parquet.column.values.ValuesReader#initFromPage(int, ByteBuffer, int)
    */
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset) throws IOException {
+  public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException {
     int effectiveBitLength = valueCount * bitsPerValue;
     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.in = new ByteBufferInputStream(in, offset, length);
     this.bitPackingReader = createBitPackingReader(bitsPerValue, this.in, valueCount);
     this.nextOffset = offset + length;
   }
-  
+
   @Override
   public int getNextOffset() {
     return nextOffset;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesWriter.java
index 24436ef..08751a0 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/BitPackingValuesWriter.java
@@ -24,6 +24,7 @@ import static org.apache.parquet.column.values.bitpacking.BitPacking.getBitPacki
 
 import java.io.IOException;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
 import org.apache.parquet.column.Encoding;
@@ -47,9 +48,9 @@ public class BitPackingValuesWriter extends ValuesWriter {
    * @param bound the maximum value stored by this column
    * @param pageSize
    */
-  public BitPackingValuesWriter(int bound, int initialCapacity, int pageSize) {
+  public BitPackingValuesWriter(int bound, int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
     this.bitsPerValue = getWidthFromMaxInt(bound);
-    this.out = new CapacityByteArrayOutputStream(initialCapacity, pageSize);
+    this.out = new CapacityByteArrayOutputStream(initialCapacity, pageSize, allocator);
     init();
   }
 
@@ -103,6 +104,11 @@ public class BitPackingValuesWriter extends ValuesWriter {
     init();
   }
 
+  @Override
+  public void close() {
+    out.close();
+  }
+
   /**
    * {@inheritDoc}
    * @see org.apache.parquet.column.values.ValuesWriter#getAllocatedSize()

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java
index c0ab7e0..f4c8c8e 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesReader.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.values.bitpacking;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesUtils;
@@ -34,7 +35,7 @@ public class ByteBitPackingValuesReader extends ValuesReader {
   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 ByteBuffer encoded;
   private int encodedPos;
   private int nextOffset;
 
@@ -47,8 +48,13 @@ public class ByteBitPackingValuesReader extends ValuesReader {
   public int readInteger() {
     ++ decodedPosition;
     if (decodedPosition == decoded.length) {
-      if (encodedPos + bitWidth > encoded.length) {
-        packer.unpack8Values(Arrays.copyOfRange(encoded, encodedPos, encodedPos + bitWidth), 0, decoded, 0);
+      encoded.position(encodedPos);
+      if (encodedPos + bitWidth > encoded.limit()) {
+        // unpack8Values needs at least bitWidth bytes to read from,
+        // We have to fill in 0 byte at the end of encoded bytes.
+        byte[] tempEncode = new byte[bitWidth];
+        encoded.get(tempEncode, 0, encoded.limit() - encodedPos);
+        packer.unpack8Values(tempEncode, 0, decoded, 0);
       } else {
         packer.unpack8Values(encoded, encodedPos, decoded, 0);
       }
@@ -59,7 +65,7 @@ public class ByteBitPackingValuesReader extends ValuesReader {
   }
 
   @Override
-  public void initFromPage(int valueCount, byte[] page, int offset)
+  public void initFromPage(int valueCount, ByteBuffer page, int offset)
       throws IOException {
     int effectiveBitLength = valueCount * bitWidth;
     int length = BytesUtils.paddedByteCountFromBits(effectiveBitLength); // ceil

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java
index d0240bb..a33e22c 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java
@@ -22,6 +22,7 @@ import static org.apache.parquet.column.Encoding.BIT_PACKED;
 
 import java.io.IOException;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.Encoding;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitReader.java
index 865eea2..caea5b5 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitReader.java
@@ -19,13 +19,14 @@
 package org.apache.parquet.column.values.boundedint;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.io.ParquetDecodingException;
 
 class BitReader {
   private int currentByte = 0;
   private int currentPosition = 8;
-  private byte[] buf;
+  private ByteBuffer buf;
   private int currentBufferPosition = 0;
   private static final int[] byteGetValueMask = new int[8];
   private static final int[] readMask = new int[32];
@@ -50,7 +51,7 @@ class BitReader {
    * The array is not copied, so must not be mutated during the course of
    * reading.
    */
-  public void prepare(byte[] buf, int offset, int length) {
+  public void prepare(ByteBuffer buf, int offset, int length) {
     this.buf = buf;
     this.endBufferPosistion = offset + length;
     currentByte = 0;
@@ -87,7 +88,7 @@ class BitReader {
 
   private int getNextByte() {
     if (currentBufferPosition < endBufferPosistion) {
-      return buf[currentBufferPosition++] & 0xFF;
+      return buf.get(currentBufferPosition++) & 0xFF;
     }
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitWriter.java
index 1d1d9d1..9489714 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BitWriter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.parquet.column.values.boundedint;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
@@ -41,8 +42,8 @@ class BitWriter {
     }
   }
 
-  public BitWriter(int initialCapacity, int pageSize) {
-    this.baos = new CapacityByteArrayOutputStream(initialCapacity, pageSize);
+  public BitWriter(int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
+    this.baos = new CapacityByteArrayOutputStream(initialCapacity, pageSize, allocator);
   }
 
   public void writeBit(boolean bit) {
@@ -156,4 +157,11 @@ class BitWriter {
   public String memUsageString(String prefix) {
     return baos.memUsageString(prefix);
   }
+
+  public void close() {
+    currentByte = 0;
+    currentBytePosition = 0;
+    finished = false;
+    baos.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesFactory.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesFactory.java b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesFactory.java
index 3ace6e2..bbbf8da 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesFactory.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesFactory.java
@@ -18,6 +18,7 @@
  */
 package org.apache.parquet.column.values.boundedint;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.ValuesWriter;
 
@@ -26,7 +27,7 @@ public abstract class BoundedIntValuesFactory {
     return bound == 0 ? new ZeroIntegerValuesReader() : new BoundedIntValuesReader(bound);
   }
 
-  public static ValuesWriter getBoundedWriter(int bound, int initialCapacity, int pageSize) {
-    return bound == 0 ? new DevNullValuesWriter() : new BoundedIntValuesWriter(bound, initialCapacity, pageSize);
+  public static ValuesWriter getBoundedWriter(int bound, int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
+    return bound == 0 ? new DevNullValuesWriter() : new BoundedIntValuesWriter(bound, initialCapacity, pageSize, allocator);
   }
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesReader.java
index dda4431..c322125 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesReader.java
@@ -21,6 +21,7 @@ package org.apache.parquet.column.values.boundedint;
 import static org.apache.parquet.Log.DEBUG;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesUtils;
@@ -70,8 +71,8 @@ class BoundedIntValuesReader extends ValuesReader {
   // bytes would have to be serialized). This is the flip-side
   // to BoundedIntColumnWriter.writeData(BytesOutput)
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset) throws IOException {
-    if (DEBUG) LOG.debug("reading size at "+ offset + ": " + in[offset] + " " + in[offset + 1] + " " + in[offset + 2] + " " + in[offset + 3] + " ");
+  public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException {
+    if (DEBUG) LOG.debug("reading size at "+ offset + ": " + in.get(offset) + " " + in.get(offset + 1) + " " + in.get(offset + 2) + " " + in.get(offset + 3) + " ");
     int totalBytes = BytesUtils.readIntLittleEndian(in, offset);
     if (DEBUG) LOG.debug("will read "+ totalBytes + " bytes");
     currentValueCt = 0;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesWriter.java
index 0acaaf7..a90a6e5 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/BoundedIntValuesWriter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.parquet.column.values.boundedint;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import static org.apache.parquet.bytes.BytesInput.concat;
 import static org.apache.parquet.column.Encoding.RLE;
 import org.apache.parquet.Log;
@@ -59,11 +60,11 @@ class BoundedIntValuesWriter extends ValuesWriter {
     }
   }
 
-  public BoundedIntValuesWriter(int bound, int initialCapacity, int pageSize) {
+  public BoundedIntValuesWriter(int bound, int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
     if (bound == 0) {
       throw new ParquetEncodingException("Value bound cannot be 0. Use DevNullColumnWriter instead.");
     }
-    this.bitWriter = new BitWriter(initialCapacity, pageSize);
+    this.bitWriter = new BitWriter(initialCapacity, pageSize, allocator);
     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);
@@ -101,6 +102,11 @@ class BoundedIntValuesWriter extends ValuesWriter {
   }
 
   @Override
+  public void close() {
+    bitWriter.close();
+  }
+
+  @Override
   public void writeInteger(int val) {
     if (currentValue == val) {
       currentValueCt++;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/ZeroIntegerValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/ZeroIntegerValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/ZeroIntegerValuesReader.java
index 9201596..8c78c38 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/ZeroIntegerValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/boundedint/ZeroIntegerValuesReader.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.column.values.boundedint;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.column.values.ValuesReader;
 
@@ -36,7 +37,7 @@ public class ZeroIntegerValuesReader extends ValuesReader {
   }
 
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset) throws IOException {
+  public void initFromPage(int valueCount, ByteBuffer in, int offset) throws IOException {
     this.nextOffset = offset;
   }
   

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
index c1678ae..3f92deb 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
@@ -19,14 +19,15 @@
 package org.apache.parquet.column.values.delta;
 
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.bitpacking.BytePacker;
 import org.apache.parquet.column.values.bitpacking.Packer;
 import org.apache.parquet.io.ParquetDecodingException;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 /**
  * Read values written by {@link DeltaBinaryPackingValuesWriter}
@@ -40,7 +41,7 @@ public class DeltaBinaryPackingValuesReader extends ValuesReader {
    */
   private int valuesRead;
   private int minDeltaInCurrentBlock;
-  private byte[] page;
+  private ByteBuffer page;
   /**
    * stores the decoded values including the first value which is written to the header
    */
@@ -50,7 +51,7 @@ public class DeltaBinaryPackingValuesReader extends ValuesReader {
    * when data is not aligned to mini block, which means padding 0s are in the buffer
    */
   private int valuesBuffered;
-  private ByteArrayInputStream in;
+  private ByteBufferInputStream in;
   private int nextOffset;
   private DeltaBinaryPackingConfig config;
   private int[] bitWidths;
@@ -64,8 +65,8 @@ public class DeltaBinaryPackingValuesReader extends ValuesReader {
    * @throws IOException
    */
   @Override
-  public void initFromPage(int valueCount, byte[] page, int offset) throws IOException {
-    in = new ByteArrayInputStream(page, offset, page.length - offset);
+  public void initFromPage(int valueCount, ByteBuffer page, int offset) throws IOException {
+    in = new ByteBufferInputStream(page, offset, page.limit() - offset);
     this.config = DeltaBinaryPackingConfig.readConfig(in);
     this.page = page;
     this.totalValueCount = BytesUtils.readUnsignedVarInt(in);
@@ -78,7 +79,7 @@ public class DeltaBinaryPackingValuesReader extends ValuesReader {
     while (valuesBuffered < totalValueCount) { //values Buffered could be more than totalValueCount, since we flush on a mini block basis
       loadNewBlockToBuffer();
     }
-    this.nextOffset = page.length - in.available();
+    this.nextOffset = page.limit() - in.available();
   }
   
   @Override
@@ -151,7 +152,7 @@ public class DeltaBinaryPackingValuesReader extends ValuesReader {
 
   private void unpack8Values(BytePacker packer) {
     //calculate the pos because the packer api uses array not stream
-    int pos = page.length - in.available();
+    int pos = page.limit() - in.available();
     packer.unpack8Values(page, pos, valuesBuffer, valuesBuffered);
     this.valuesBuffered += 8;
     //sync the pos in stream

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java
index 1b3692f..421182f 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.parquet.column.values.delta;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
@@ -111,16 +112,16 @@ public class DeltaBinaryPackingValuesWriter extends ValuesWriter {
    */
   private int minDeltaInCurrentBlock = Integer.MAX_VALUE;
 
-  public DeltaBinaryPackingValuesWriter(int slabSize, int pageSize) {
-    this(DEFAULT_NUM_BLOCK_VALUES, DEFAULT_NUM_MINIBLOCKS, slabSize, pageSize);
+  public DeltaBinaryPackingValuesWriter(int slabSize, int pageSize, ByteBufferAllocator allocator) {
+    this(DEFAULT_NUM_BLOCK_VALUES, DEFAULT_NUM_MINIBLOCKS, slabSize, pageSize, allocator);
   }
 
-  public DeltaBinaryPackingValuesWriter(int blockSizeInValues, int miniBlockNum, int slabSize, int pageSize) {
+  public DeltaBinaryPackingValuesWriter(int blockSizeInValues, int miniBlockNum, int slabSize, int pageSize, ByteBufferAllocator allocator) {
     this.config = new DeltaBinaryPackingConfig(blockSizeInValues, miniBlockNum);
     bitWidths = new int[config.miniBlockNumInABlock];
     deltaBlockBuffer = new int[blockSizeInValues];
     miniBlockByteBuffer = new byte[config.miniBlockSizeInValues * MAX_BITWIDTH];
-    baos = new CapacityByteArrayOutputStream(slabSize, pageSize);
+    baos = new CapacityByteArrayOutputStream(slabSize, pageSize, allocator);
   }
 
   @Override
@@ -258,6 +259,14 @@ public class DeltaBinaryPackingValuesWriter extends ValuesWriter {
   }
 
   @Override
+  public void close() {
+    this.totalValueCount = 0;
+    this.baos.close();
+    this.deltaValuesToFlush = 0;
+    this.minDeltaInCurrentBlock = Integer.MAX_VALUE;
+  }
+
+  @Override
   public long getAllocatedSize() {
     return baos.getCapacity();
   }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
index fb9bdc5..41f221d 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
@@ -21,6 +21,7 @@ package org.apache.parquet.column.values.deltalengthbytearray;
 import static org.apache.parquet.Log.DEBUG;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.Log;
 import org.apache.parquet.column.values.ValuesReader;
@@ -28,7 +29,7 @@ import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader;
 import org.apache.parquet.io.api.Binary;
 
 /**
- * Reads binary data written by {@link DeltaLengthByteArrayValueWriter}
+ * Reads binary data written by {@link DeltaLengthByteArrayValuesWriter}
  *
  * @author Aniket Mokashi
  *
@@ -37,7 +38,7 @@ public class DeltaLengthByteArrayValuesReader extends ValuesReader {
 
   private static final Log LOG = Log.getLog(DeltaLengthByteArrayValuesReader.class);
   private ValuesReader lengthReader;
-  private byte[] in;
+  private ByteBuffer in;
   private int offset;
 
   public DeltaLengthByteArrayValuesReader() {
@@ -45,9 +46,9 @@ public class DeltaLengthByteArrayValuesReader extends ValuesReader {
   }
 
   @Override
-  public void initFromPage(int valueCount, byte[] in, int offset)
+  public void initFromPage(int valueCount, ByteBuffer in, int offset)
       throws IOException {
-    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.length - offset));
+    if (DEBUG) LOG.debug("init from page at offset "+ offset + " for length " + (in.limit() - offset));
     lengthReader.initFromPage(valueCount, in, offset);
     offset = lengthReader.getNextOffset();
     this.in = in;
@@ -59,7 +60,7 @@ public class DeltaLengthByteArrayValuesReader extends ValuesReader {
     int length = lengthReader.readInteger();
     int start = offset;
     offset = start + length;
-    return Binary.fromConstantByteArray(in, start, length);
+    return Binary.fromConstantByteBuffer(in, start, length);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java
index 3f686cc..2d6b213 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java
@@ -20,6 +20,7 @@ package org.apache.parquet.column.values.deltalengthbytearray;
 
 import java.io.IOException;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
@@ -48,13 +49,13 @@ public class DeltaLengthByteArrayValuesWriter extends ValuesWriter {
   private CapacityByteArrayOutputStream arrayOut;
   private LittleEndianDataOutputStream out;
 
-  public DeltaLengthByteArrayValuesWriter(int initialSize, int pageSize) {
-    arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize);
+  public DeltaLengthByteArrayValuesWriter(int initialSize, int pageSize, ByteBufferAllocator allocator) {
+    arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize, allocator);
     out = new LittleEndianDataOutputStream(arrayOut);
     lengthWriter = new DeltaBinaryPackingValuesWriter(
         DeltaBinaryPackingValuesWriter.DEFAULT_NUM_BLOCK_VALUES,
         DeltaBinaryPackingValuesWriter.DEFAULT_NUM_MINIBLOCKS,
-        initialSize, pageSize);
+        initialSize, pageSize, allocator);
   }
 
   @Override
@@ -95,6 +96,12 @@ public class DeltaLengthByteArrayValuesWriter extends ValuesWriter {
   }
 
   @Override
+  public void close() {
+    lengthWriter.close();
+    arrayOut.close();
+  }
+
+  @Override
   public long getAllocatedSize() {
     return lengthWriter.getAllocatedSize() + arrayOut.getCapacity();
   }
@@ -104,3 +111,4 @@ public class DeltaLengthByteArrayValuesWriter extends ValuesWriter {
     return arrayOut.memUsageString(lengthWriter.memUsageString(prefix) + " DELTA_LENGTH_BYTE_ARRAY");
   }
 }
+

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayReader.java
index 87ec08e..c2cfd6d 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayReader.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.column.values.deltastrings;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.parquet.column.values.RequiresPreviousReader;
 import org.apache.parquet.column.values.ValuesReader;
@@ -45,13 +46,13 @@ public class DeltaByteArrayReader extends ValuesReader implements RequiresPrevio
   }
 
   @Override
-  public void initFromPage(int valueCount, byte[] page, int offset)
+  public void initFromPage(int valueCount, ByteBuffer page, int offset)
       throws IOException {
     prefixLengthReader.initFromPage(valueCount, page, offset);
     int next = prefixLengthReader.getNextOffset();
     suffixReader.initFromPage(valueCount, page, next);	
   }
-
+  
   @Override
   public void skip() {
     prefixLengthReader.skip();

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayWriter.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayWriter.java
index 54234db..1604ddb 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayWriter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.parquet.column.values.deltastrings;
 
+import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.values.ValuesWriter;
@@ -41,9 +42,9 @@ public class DeltaByteArrayWriter extends ValuesWriter{
   private ValuesWriter suffixWriter;
   private byte[] previous;
 
-  public DeltaByteArrayWriter(int initialCapacity, int pageSize) {
-    this.prefixLengthWriter = new DeltaBinaryPackingValuesWriter(128, 4, initialCapacity, pageSize);
-    this.suffixWriter = new DeltaLengthByteArrayValuesWriter(initialCapacity, pageSize);
+  public DeltaByteArrayWriter(int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
+    this.prefixLengthWriter = new DeltaBinaryPackingValuesWriter(128, 4, initialCapacity, pageSize, allocator);
+    this.suffixWriter = new DeltaLengthByteArrayValuesWriter(initialCapacity, pageSize, allocator);
     this.previous = new byte[0];
   }
 
@@ -70,6 +71,12 @@ public class DeltaByteArrayWriter extends ValuesWriter{
   }
 
   @Override
+  public void close() {
+    prefixLengthWriter.close();
+    suffixWriter.close();
+  }
+
+  @Override
   public long getAllocatedSize() {
     return prefixLengthWriter.getAllocatedSize() + suffixWriter.getAllocatedSize();
   }
@@ -85,6 +92,7 @@ public class DeltaByteArrayWriter extends ValuesWriter{
     int i = 0;
     byte[] vb = v.getBytes();
     int length = previous.length < vb.length ? previous.length : vb.length;
+    // find the number of matching prefix bytes between this value and the previous one
     for(i = 0; (i < length) && (previous[i] == vb[i]); i++);
     prefixLengthWriter.writeInteger(i);
     suffixWriter.writeBytes(v.slice(i, vb.length - i));

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/6b605a4e/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java
index 972c87e..e421da9 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesReader.java
@@ -20,9 +20,10 @@ package org.apache.parquet.column.values.dictionary;
 
 import static org.apache.parquet.Log.DEBUG;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
+import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.Log;
 import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.Dictionary;
@@ -40,7 +41,7 @@ import org.apache.parquet.io.api.Binary;
 public class DictionaryValuesReader extends ValuesReader {
   private static final Log LOG = Log.getLog(DictionaryValuesReader.class);
 
-  private ByteArrayInputStream in;
+  private ByteBufferInputStream in;
 
   private Dictionary dictionary;
 
@@ -51,12 +52,12 @@ public class DictionaryValuesReader extends ValuesReader {
   }
 
   @Override
-  public void initFromPage(int valueCount, byte[] page, int offset)
+  public void initFromPage(int valueCount, ByteBuffer page, int offset)
       throws IOException {
-    this.in = new ByteArrayInputStream(page, offset, page.length - offset);
-    if (page.length - offset > 0) {
+    this.in = new ByteBufferInputStream(page, offset, page.limit() - offset);
+    if (page.limit() - offset > 0) {
       if (DEBUG)
-        LOG.debug("init from page at offset " + offset + " for length " + (page.length - offset));
+        LOG.debug("init from page at offset " + offset + " for length " + (page.limit() - offset));
       int bitWidth = BytesUtils.readIntLittleEndianOnOneByte(in);
       if (DEBUG) LOG.debug("bit width " + bitWidth);
       decoder = new RunLengthBitPackingHybridDecoder(bitWidth, in);