parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From b...@apache.org
Subject [19/51] [partial] parquet-mr git commit: PARQUET-23: Rename to org.apache.parquet.
Date Mon, 27 Apr 2015 23:12:16 GMT
http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/parquet/bytes/BytesUtils.java b/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
deleted file mode 100644
index 4413ced..0000000
--- a/parquet-common/src/main/java/parquet/bytes/BytesUtils.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/* 
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package parquet.bytes;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import parquet.Log;
-
-/**
- * utility methods to deal with bytes
- *
- * @author Julien Le Dem
- *
- */
-public class BytesUtils {
-  private static final Log LOG = Log.getLog(BytesUtils.class);
-
-  public static final Charset UTF8 = Charset.forName("UTF-8");
-
-  /**
-   * give the number of bits needed to encode an int given the max value
-   * @param bound max int that we want to encode
-   * @return the number of bits required
-   */
-  public static int getWidthFromMaxInt(int bound) {
-    return 32 - Integer.numberOfLeadingZeros(bound);
-  }
-
-  /**
-   * reads an int in little endian at the given position
-   * @param in
-   * @param offset
-   * @return
-   * @throws IOException
-   */
-  public static int readIntLittleEndian(byte[] in, int offset) throws IOException {
-    int ch4 = in[offset] & 0xff;
-    int ch3 = in[offset + 1] & 0xff;
-    int ch2 = in[offset + 2] & 0xff;
-    int ch1 = in[offset + 3] & 0xff;
-    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
-  }
-
-  public static int readIntLittleEndian(InputStream in) throws IOException {
-    // TODO: this is duplicated code in LittleEndianDataInputStream
-    int ch1 = in.read();
-    int ch2 = in.read();
-    int ch3 = in.read();
-    int ch4 = in.read();
-    if ((ch1 | ch2 | ch3 | ch4) < 0) {
-        throw new EOFException();
-    }
-    return ((ch4 << 24) + (ch3 << 16) + (ch2 << 8) + (ch1 << 0));
-  }
-
-  public static int readIntLittleEndianOnOneByte(InputStream in) throws IOException {
-      int ch1 = in.read();
-      if (ch1 < 0) {
-        throw new EOFException();
-      }
-      return ch1;
-  }
-
-  public static int readIntLittleEndianOnTwoBytes(InputStream in) throws IOException {
-      int ch1 = in.read();
-      int ch2 = in.read();
-      if ((ch1 | ch2 ) < 0) {
-          throw new EOFException();
-      }
-      return ((ch2 << 8) + (ch1 << 0));
-  }
-
-  public static int readIntLittleEndianOnThreeBytes(InputStream in) throws IOException {
-      int ch1 = in.read();
-      int ch2 = in.read();
-      int ch3 = in.read();
-      if ((ch1 | ch2 | ch3 ) < 0) {
-          throw new EOFException();
-      }
-      return ((ch3 << 16) + (ch2 << 8) + (ch1 << 0));
-  }
-
-  public static int readIntLittleEndianPaddedOnBitWidth(InputStream in, int bitWidth)
-      throws IOException {
-
-    int bytesWidth = paddedByteCountFromBits(bitWidth);
-    switch (bytesWidth) {
-      case 0:
-        return 0;
-      case 1:
-        return BytesUtils.readIntLittleEndianOnOneByte(in);
-      case 2:
-        return BytesUtils.readIntLittleEndianOnTwoBytes(in);
-      case 3:
-        return  BytesUtils.readIntLittleEndianOnThreeBytes(in);
-      case 4:
-        return BytesUtils.readIntLittleEndian(in);
-      default:
-        throw new IOException(
-          String.format("Encountered bitWidth (%d) that requires more than 4 bytes", bitWidth));
-    }
-  }
-
-  public static void writeIntLittleEndianOnOneByte(OutputStream out, int v) throws IOException {
-    out.write((v >>>  0) & 0xFF);
-  }
-
-  public static void writeIntLittleEndianOnTwoBytes(OutputStream out, int v) throws IOException {
-    out.write((v >>>  0) & 0xFF);
-    out.write((v >>>  8) & 0xFF);
-  }
-
-  public static void writeIntLittleEndianOnThreeBytes(OutputStream out, int v) throws IOException {
-    out.write((v >>>  0) & 0xFF);
-    out.write((v >>>  8) & 0xFF);
-    out.write((v >>> 16) & 0xFF);
-  }
-
-  public static void writeIntLittleEndian(OutputStream out, int v) throws IOException {
-    // TODO: this is duplicated code in LittleEndianDataOutputStream
-    out.write((v >>>  0) & 0xFF);
-    out.write((v >>>  8) & 0xFF);
-    out.write((v >>> 16) & 0xFF);
-    out.write((v >>> 24) & 0xFF);
-    if (Log.DEBUG) LOG.debug("write le int: " + v + " => "+ ((v >>>  0) & 0xFF) + " " + ((v >>>  8) & 0xFF) + " " + ((v >>> 16) & 0xFF) + " " + ((v >>> 24) & 0xFF));
-  }
-
-  /**
-   * Write a little endian int to out, using the the number of bytes required by
-   * bit width
-   */
-  public static void writeIntLittleEndianPaddedOnBitWidth(OutputStream out, int v, int bitWidth)
-      throws IOException {
-
-    int bytesWidth = paddedByteCountFromBits(bitWidth);
-    switch (bytesWidth) {
-      case 0:
-        break;
-      case 1:
-        writeIntLittleEndianOnOneByte(out, v);
-        break;
-      case 2:
-        writeIntLittleEndianOnTwoBytes(out, v);
-        break;
-      case 3:
-        writeIntLittleEndianOnThreeBytes(out, v);
-        break;
-      case 4:
-        writeIntLittleEndian(out, v);
-        break;
-      default:
-        throw new IOException(
-          String.format("Encountered value (%d) that requires more than 4 bytes", v));
-    }
-  }
-
-  public static int readUnsignedVarInt(InputStream in) throws IOException {
-    int value = 0;
-    int i = 0;
-    int b;
-    while (((b = in.read()) & 0x80) != 0) {
-      value |= (b & 0x7F) << i;
-      i += 7;
-    }
-    return value | (b << i);
-  }
-
-  /**
-   * uses a trick mentioned in https://developers.google.com/protocol-buffers/docs/encoding to read zigZag encoded data
-   * @param in
-   * @return
-   * @throws IOException
-   */
-  public static int readZigZagVarInt(InputStream in) throws IOException {
-    int raw = readUnsignedVarInt(in);
-    int temp = (((raw << 31) >> 31) ^ raw) >> 1;
-    return temp ^ (raw & (1 << 31));
-  }
-
-  public static void writeUnsignedVarInt(int value, OutputStream out) throws IOException {
-    while ((value & 0xFFFFFF80) != 0L) {
-      out.write((value & 0x7F) | 0x80);
-      value >>>= 7;
-    }
-    out.write(value & 0x7F);
-  }
-
-  public static void writeZigZagVarInt(int intValue, OutputStream out) throws IOException{
-    writeUnsignedVarInt((intValue << 1) ^ (intValue >> 31), out);
-  }
-
-  /**
-   * @param bitLength a count of bits
-   * @return the corresponding byte count padded to the next byte
-   */
-  public static int paddedByteCountFromBits(int bitLength) {
-    return (bitLength + 7) / 8;
-  }
-
-  public static byte[] intToBytes(int value) {
-    byte[] outBuffer = new byte[4];
-    outBuffer[3] = (byte)(value >>> 24);
-    outBuffer[2] = (byte)(value >>> 16);
-    outBuffer[1] = (byte)(value >>>  8);
-    outBuffer[0] = (byte)(value >>>  0);
-    return outBuffer;
-  }
-
-  public static int bytesToInt(byte[] bytes) {
-    return ((int)(bytes[3] & 255) << 24) +
-           ((int)(bytes[2] & 255) << 16) +
-           ((int)(bytes[1] & 255) <<  8) +
-           ((int)(bytes[0] & 255) <<  0);
-  }
-
-  public static byte[] longToBytes(long value) {
-    byte[] outBuffer = new byte[8];
-    outBuffer[7] = (byte)(value >>> 56);
-    outBuffer[6] = (byte)(value >>> 48);
-    outBuffer[5] = (byte)(value >>> 40);
-    outBuffer[4] = (byte)(value >>> 32);
-    outBuffer[3] = (byte)(value >>> 24);
-    outBuffer[2] = (byte)(value >>> 16);
-    outBuffer[1] = (byte)(value >>>  8);
-    outBuffer[0] = (byte)(value >>>  0);
-    return outBuffer;
-  }
-
-  public static long bytesToLong(byte[] bytes) {
-    return (((long)bytes[7] << 56) +
-           ((long)(bytes[6] & 255) << 48) +
-           ((long)(bytes[5] & 255) << 40) +
-           ((long)(bytes[4] & 255) << 32) +
-           ((long)(bytes[3] & 255) << 24) +
-           ((long)(bytes[2] & 255) << 16) +
-           ((long)(bytes[1] & 255) <<  8) +
-           ((long)(bytes[0] & 255) <<  0));
-  }
-
-  public static byte[] booleanToBytes(boolean value) {
-    byte[] outBuffer = new byte[1];
-    outBuffer[0] = (byte)(value ? 1 : 0);
-    return outBuffer;
-  }
-
-  public static boolean bytesToBool(byte[] bytes) {
-    return ((int)(bytes[0] & 255) != 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/main/java/parquet/hadoop/metadata/Canonicalizer.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/parquet/hadoop/metadata/Canonicalizer.java b/parquet-common/src/main/java/parquet/hadoop/metadata/Canonicalizer.java
deleted file mode 100644
index 8208252..0000000
--- a/parquet-common/src/main/java/parquet/hadoop/metadata/Canonicalizer.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/* 
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package parquet.hadoop.metadata;
-
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * returns canonical representation of objects (similar to String.intern()) to save memory
- * if a.equals(b) then canonicalize(a) == canonicalize(b)
- * this class is thread safe
- * @author Julien Le Dem
- *
- * @param <T>
- */
-public class Canonicalizer<T> {
-
-  private ConcurrentHashMap<T, T> canonicals = new ConcurrentHashMap<T, T>();
-
-  /**
-   * @param value the value to canonicalize
-   * @return the corresponding canonical value
-   */
-  final public T canonicalize(T value) {
-    T canonical = canonicals.get(value);
-    if (canonical == null) {
-      value = toCanonical(value);
-      T existing = canonicals.putIfAbsent(value, value);
-      // putIfAbsent is atomic, making sure we always return the same canonical representation of the value
-      if (existing == null) {
-        canonical = value;
-      } else {
-        canonical = existing;
-      }
-    }
-    return canonical;
-  }
-
-  /**
-   * @param value the value to canonicalize if needed
-   * @return the canonicalized value
-   */
-  protected T toCanonical(T value) {
-    return value;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/main/java/parquet/hadoop/metadata/ColumnPath.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/main/java/parquet/hadoop/metadata/ColumnPath.java b/parquet-common/src/main/java/parquet/hadoop/metadata/ColumnPath.java
deleted file mode 100644
index 13375f6..0000000
--- a/parquet-common/src/main/java/parquet/hadoop/metadata/ColumnPath.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/* 
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package parquet.hadoop.metadata;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Iterator;
-
-import static parquet.Preconditions.checkNotNull;
-
-public final class ColumnPath implements Iterable<String>, Serializable {
-
-  private static Canonicalizer<ColumnPath> paths = new Canonicalizer<ColumnPath>() {
-    @Override
-    protected ColumnPath toCanonical(ColumnPath value) {
-      String[] path = new String[value.p.length];
-      for (int i = 0; i < value.p.length; i++) {
-        path[i] = value.p[i].intern();
-      }
-      return new ColumnPath(path);
-    }
-  };
-
-  public static ColumnPath fromDotString(String path) {
-    checkNotNull(path, "path");
-    return get(path.split("\\."));
-  }
-
-  public static ColumnPath get(String... path){
-    return paths.canonicalize(new ColumnPath(path));
-  }
-
-  private final String[] p;
-
-  private ColumnPath(String[] path) {
-    this.p = path;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof ColumnPath) {
-      return Arrays.equals(p, ((ColumnPath)obj).p);
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return Arrays.hashCode(p);
-  }
-
-  public String toDotString() {
-    Iterator<String> iter = Arrays.asList(p).iterator();
-    StringBuilder sb = new StringBuilder();
-    while (iter.hasNext()) {
-      sb.append(iter.next());
-      if (iter.hasNext()) {
-        sb.append('.');
-      }
-    }
-    return sb.toString();
-  }
-
-  @Override
-  public String toString() {
-    return Arrays.toString(p);
-  }
-
-  @Override
-  public Iterator<String> iterator() {
-    return Arrays.asList(p).iterator();
-  }
-
-  public int size() {
-    return p.length;
-  }
-
-  public String[] toArray() {
-    return p;
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/test/java/org/apache/parquet/TestLog.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/test/java/org/apache/parquet/TestLog.java b/parquet-common/src/test/java/org/apache/parquet/TestLog.java
new file mode 100644
index 0000000..4508b0d
--- /dev/null
+++ b/parquet-common/src/test/java/org/apache/parquet/TestLog.java
@@ -0,0 +1,31 @@
+/* 
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestLog {
+
+  @Test
+  public void test() {
+    // Use a compile time log level of INFO for performance
+    Assert.assertFalse("Do not merge in log level DEBUG", Log.DEBUG);
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/test/java/org/apache/parquet/TestPreconditions.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/test/java/org/apache/parquet/TestPreconditions.java b/parquet-common/src/test/java/org/apache/parquet/TestPreconditions.java
new file mode 100644
index 0000000..d4b770b
--- /dev/null
+++ b/parquet-common/src/test/java/org/apache/parquet/TestPreconditions.java
@@ -0,0 +1,58 @@
+/**
+ * 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 org.junit.Assert;
+import org.junit.Test;
+
+public class TestPreconditions {
+  @Test
+  public void testCheckArgument() {
+    try {
+      Preconditions.checkArgument(true, "Test message: %s %s", 12, null);
+    } catch (IllegalArgumentException e) {
+      Assert.fail("Should not throw exception when isValid is true");
+    }
+
+    try {
+      Preconditions.checkArgument(false, "Test message: %s %s", 12, null);
+      Assert.fail("Should throw exception when isValid is false");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals("Should format message",
+          "Test message: 12 null", e.getMessage());
+    }
+  }
+
+  @Test
+  public void testCheckState() {
+    try {
+      Preconditions.checkState(true, "Test message: %s %s", 12, null);
+    } catch (IllegalStateException e) {
+      Assert.fail("Should not throw exception when isValid is true");
+    }
+
+    try {
+      Preconditions.checkState(false, "Test message: %s %s", 12, null);
+      Assert.fail("Should throw exception when isValid is false");
+    } catch (IllegalStateException e) {
+      Assert.assertEquals("Should format message",
+          "Test message: 12 null", e.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/test/java/org/apache/parquet/bytes/TestBytesUtil.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/test/java/org/apache/parquet/bytes/TestBytesUtil.java b/parquet-common/src/test/java/org/apache/parquet/bytes/TestBytesUtil.java
new file mode 100644
index 0000000..e8eda65
--- /dev/null
+++ b/parquet-common/src/test/java/org/apache/parquet/bytes/TestBytesUtil.java
@@ -0,0 +1,49 @@
+/* 
+ * 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 static org.junit.Assert.assertEquals;
+import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt;
+
+import org.junit.Test;
+
+public class TestBytesUtil {
+
+  @Test
+  public void testWidth() {
+    assertEquals(0, getWidthFromMaxInt(0));
+    assertEquals(1, getWidthFromMaxInt(1));
+    assertEquals(2, getWidthFromMaxInt(2));
+    assertEquals(2, getWidthFromMaxInt(3));
+    assertEquals(3, getWidthFromMaxInt(4));
+    assertEquals(3, getWidthFromMaxInt(5));
+    assertEquals(3, getWidthFromMaxInt(6));
+    assertEquals(3, getWidthFromMaxInt(7));
+    assertEquals(4, getWidthFromMaxInt(8));
+    assertEquals(4, getWidthFromMaxInt(15));
+    assertEquals(5, getWidthFromMaxInt(16));
+    assertEquals(5, getWidthFromMaxInt(31));
+    assertEquals(6, getWidthFromMaxInt(32));
+    assertEquals(6, getWidthFromMaxInt(63));
+    assertEquals(7, getWidthFromMaxInt(64));
+    assertEquals(7, getWidthFromMaxInt(127));
+    assertEquals(8, getWidthFromMaxInt(128));
+    assertEquals(8, getWidthFromMaxInt(255));
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/test/java/parquet/TestLog.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/test/java/parquet/TestLog.java b/parquet-common/src/test/java/parquet/TestLog.java
deleted file mode 100644
index 33b5038..0000000
--- a/parquet-common/src/test/java/parquet/TestLog.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/* 
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package parquet;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestLog {
-
-  @Test
-  public void test() {
-    // Use a compile time log level of INFO for performance
-    Assert.assertFalse("Do not merge in log level DEBUG", Log.DEBUG);
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/test/java/parquet/TestPreconditions.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/test/java/parquet/TestPreconditions.java b/parquet-common/src/test/java/parquet/TestPreconditions.java
deleted file mode 100644
index ffcd00f..0000000
--- a/parquet-common/src/test/java/parquet/TestPreconditions.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package parquet;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestPreconditions {
-  @Test
-  public void testCheckArgument() {
-    try {
-      Preconditions.checkArgument(true, "Test message: %s %s", 12, null);
-    } catch (IllegalArgumentException e) {
-      Assert.fail("Should not throw exception when isValid is true");
-    }
-
-    try {
-      Preconditions.checkArgument(false, "Test message: %s %s", 12, null);
-      Assert.fail("Should throw exception when isValid is false");
-    } catch (IllegalArgumentException e) {
-      Assert.assertEquals("Should format message",
-          "Test message: 12 null", e.getMessage());
-    }
-  }
-
-  @Test
-  public void testCheckState() {
-    try {
-      Preconditions.checkState(true, "Test message: %s %s", 12, null);
-    } catch (IllegalStateException e) {
-      Assert.fail("Should not throw exception when isValid is true");
-    }
-
-    try {
-      Preconditions.checkState(false, "Test message: %s %s", 12, null);
-      Assert.fail("Should throw exception when isValid is false");
-    } catch (IllegalStateException e) {
-      Assert.assertEquals("Should format message",
-          "Test message: 12 null", e.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-common/src/test/java/parquet/bytes/TestBytesUtil.java
----------------------------------------------------------------------
diff --git a/parquet-common/src/test/java/parquet/bytes/TestBytesUtil.java b/parquet-common/src/test/java/parquet/bytes/TestBytesUtil.java
deleted file mode 100644
index d449dfb..0000000
--- a/parquet-common/src/test/java/parquet/bytes/TestBytesUtil.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/* 
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package parquet.bytes;
-
-import static org.junit.Assert.assertEquals;
-import static parquet.bytes.BytesUtils.getWidthFromMaxInt;
-
-import org.junit.Test;
-
-public class TestBytesUtil {
-
-  @Test
-  public void testWidth() {
-    assertEquals(0, getWidthFromMaxInt(0));
-    assertEquals(1, getWidthFromMaxInt(1));
-    assertEquals(2, getWidthFromMaxInt(2));
-    assertEquals(2, getWidthFromMaxInt(3));
-    assertEquals(3, getWidthFromMaxInt(4));
-    assertEquals(3, getWidthFromMaxInt(5));
-    assertEquals(3, getWidthFromMaxInt(6));
-    assertEquals(3, getWidthFromMaxInt(7));
-    assertEquals(4, getWidthFromMaxInt(8));
-    assertEquals(4, getWidthFromMaxInt(15));
-    assertEquals(5, getWidthFromMaxInt(16));
-    assertEquals(5, getWidthFromMaxInt(31));
-    assertEquals(6, getWidthFromMaxInt(32));
-    assertEquals(6, getWidthFromMaxInt(63));
-    assertEquals(7, getWidthFromMaxInt(64));
-    assertEquals(7, getWidthFromMaxInt(127));
-    assertEquals(8, getWidthFromMaxInt(128));
-    assertEquals(8, getWidthFromMaxInt(255));
-  }
-}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-encoding/pom.xml
----------------------------------------------------------------------
diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml
index a3a5349..b27c77a 100644
--- a/parquet-encoding/pom.xml
+++ b/parquet-encoding/pom.xml
@@ -18,7 +18,7 @@
   -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <parent>
-    <groupId>com.twitter</groupId>
+    <groupId>org.apache.parquet</groupId>
     <artifactId>parquet</artifactId>
     <relativePath>../pom.xml</relativePath>
     <version>1.7.0-incubating-SNAPSHOT</version>
@@ -37,12 +37,12 @@
 
   <dependencies>
     <dependency>
-      <groupId>com.twitter</groupId>
+      <groupId>org.apache.parquet</groupId>
       <artifactId>parquet-common</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>com.twitter</groupId>
+      <groupId>org.apache.parquet</groupId>
       <artifactId>parquet-generator</artifactId>
       <version>${project.version}</version>
     </dependency>
@@ -76,7 +76,7 @@
           </execution>
         </executions>
         <configuration>
-          <mainClass>parquet.encoding.Generator</mainClass>          
+          <mainClass>org.apache.parquet.encoding.Generator</mainClass>
           <arguments>
             <argument>${basedir}/target/generated-src</argument>
           </arguments>

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/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
new file mode 100644
index 0000000..ac334ae
--- /dev/null
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/BytesInput.java
@@ -0,0 +1,365 @@
+/* 
+ * 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.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.Log;
+
+
+/**
+ * A source of bytes capable of writing itself to an output.
+ * A BytesInput should be consumed right away.
+ * It is not a container.
+ * For example if it is referring to a stream,
+ * subsequent BytesInput reads from the stream will be incorrect
+ * if the previous has not been consumed.
+ *
+ * @author Julien Le Dem
+ *
+ */
+abstract public class BytesInput {
+  private static final Log LOG = Log.getLog(BytesInput.class);
+  private static final boolean DEBUG = false;//Log.DEBUG;
+  private static final EmptyBytesInput EMPTY_BYTES_INPUT = new EmptyBytesInput();
+
+  /**
+   * logically concatenate the provided inputs
+   * @param inputs the inputs to concatenate
+   * @return a concatenated input
+   */
+  public static BytesInput concat(BytesInput... inputs) {
+    return new SequenceBytesIn(Arrays.asList(inputs));
+  }
+
+  /**
+   * logically concatenate the provided inputs
+   * @param inputs the inputs to concatenate
+   * @return a concatenated input
+   */
+  public static BytesInput concat(List<BytesInput> inputs) {
+    return new SequenceBytesIn(inputs);
+  }
+
+  /**
+   * @param in
+   * @param bytes number of bytes to read
+   * @return a BytesInput that will read that number of bytes from the stream
+   */
+  public static BytesInput from(InputStream in, int bytes) {
+    return new StreamBytesInput(in, bytes);
+  }
+
+  /**
+   *
+   * @param in
+   * @return a Bytes input that will write the given bytes
+   */
+  public static BytesInput from(byte[] in) {
+    if (DEBUG) LOG.debug("BytesInput from array of " + in.length + " bytes");
+    return new ByteArrayBytesInput(in, 0 , in.length);
+  }
+
+  public static BytesInput from(byte[] in, int offset, int length) {
+    if (DEBUG) LOG.debug("BytesInput from array of " + length + " bytes");
+    return new ByteArrayBytesInput(in, offset, length);
+  }
+
+  /**
+   * @param intValue the int to write
+   * @return a BytesInput that will write 4 bytes in little endian
+   */
+  public static BytesInput fromInt(int intValue) {
+    return new IntBytesInput(intValue);
+  }
+
+  /**
+   * @param intValue the int to write
+   * @return a BytesInput that will write var int
+   */
+  public static BytesInput fromUnsignedVarInt(int intValue) {
+    return new UnsignedVarIntBytesInput(intValue);
+  }
+
+  /**
+   *
+   * @param intValue the int to write
+   */
+  public static BytesInput fromZigZagVarInt(int intValue) {
+    int zigZag = (intValue << 1) ^ (intValue >> 31);
+    return new UnsignedVarIntBytesInput(zigZag);
+  }
+
+  /**
+   * @param arrayOut
+   * @return a BytesInput that will write the content of the buffer
+   */
+  public static BytesInput from(CapacityByteArrayOutputStream arrayOut) {
+    return new CapacityBAOSBytesInput(arrayOut);
+  }
+
+  /**
+   * @param arrayOut
+   * @return a BytesInput that will write the content of the buffer
+   */
+  public static BytesInput from(ByteArrayOutputStream baos) {
+    return new BAOSBytesInput(baos);
+  }
+
+  /**
+   * @return an empty bytes input
+   */
+  public static BytesInput empty() {
+    return EMPTY_BYTES_INPUT;
+  }
+
+  /**
+   * copies the input into a new byte array
+   * @param bytesInput
+   * @return
+   * @throws IOException
+   */
+  public static BytesInput copy(BytesInput bytesInput) throws IOException {
+    return from(bytesInput.toByteArray());
+  }
+
+  /**
+   * writes the bytes into a stream
+   * @param out
+   * @throws IOException
+   */
+  abstract public void writeAllTo(OutputStream out) throws IOException;
+
+  /**
+   *
+   * @return a new byte array materializing the contents of this input
+   * @throws IOException
+   */
+  public byte[] toByteArray() throws IOException {
+    BAOS baos = new BAOS((int)size());
+    this.writeAllTo(baos);
+    if (DEBUG) LOG.debug("converted " + size() + " to byteArray of " + baos.size() + " bytes");
+    return baos.getBuf();
+  }
+
+  /**
+   *
+   * @return the size in bytes that would be written
+   */
+  abstract public long size();
+
+  private static final class BAOS extends ByteArrayOutputStream {
+    private BAOS(int size) {
+      super(size);
+    }
+
+    public byte[] getBuf() {
+      return this.buf;
+    }
+  }
+
+  private static class StreamBytesInput extends BytesInput {
+    private static final Log LOG = Log.getLog(BytesInput.StreamBytesInput.class);
+    private final InputStream in;
+    private final int byteCount;
+
+    private StreamBytesInput(InputStream in, int byteCount) {
+      super();
+      this.in = in;
+      this.byteCount = byteCount;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      if (DEBUG) LOG.debug("write All "+ byteCount + " bytes");
+      // TODO: more efficient
+      out.write(this.toByteArray());
+    }
+
+    public byte[] toByteArray() throws IOException {
+      if (DEBUG) LOG.debug("read all "+ byteCount + " bytes");
+      byte[] buf = new byte[byteCount];
+      new DataInputStream(in).readFully(buf);
+      return buf;
+    }
+
+    @Override
+    public long size() {
+      return byteCount;
+    }
+
+  }
+
+  private static class SequenceBytesIn extends BytesInput {
+    private static final Log LOG = Log.getLog(BytesInput.SequenceBytesIn.class);
+
+    private final List<BytesInput> inputs;
+    private final long size;
+
+    private SequenceBytesIn(List<BytesInput> inputs) {
+      this.inputs = inputs;
+      long total = 0;
+      for (BytesInput input : inputs) {
+        total += input.size();
+      }
+      this.size = total;
+    }
+
+    @SuppressWarnings("unused")
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      for (BytesInput input : inputs) {
+        if (DEBUG) LOG.debug("write " + input.size() + " bytes to out");
+        if (DEBUG && input instanceof SequenceBytesIn) LOG.debug("{");
+        input.writeAllTo(out);
+        if (DEBUG && input instanceof SequenceBytesIn) LOG.debug("}");
+      }
+    }
+
+    @Override
+    public long size() {
+      return size;
+    }
+
+  }
+
+  private static class IntBytesInput extends BytesInput {
+
+    private final int intValue;
+
+    public IntBytesInput(int intValue) {
+      this.intValue = intValue;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      BytesUtils.writeIntLittleEndian(out, intValue);
+    }
+
+    @Override
+    public long size() {
+      return 4;
+    }
+
+  }
+
+  private static class UnsignedVarIntBytesInput extends BytesInput {
+
+    private final int intValue;
+
+    public UnsignedVarIntBytesInput(int intValue) {
+      this.intValue = intValue;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      BytesUtils.writeUnsignedVarInt(intValue, out);
+    }
+
+    @Override
+    public long size() {
+      int s = 5 - ((Integer.numberOfLeadingZeros(intValue) + 3) / 7);
+      return s == 0 ? 1 : s;
+    }
+  }
+
+  private static class EmptyBytesInput extends BytesInput {
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+    }
+
+    @Override
+    public long size() {
+      return 0;
+    }
+
+  }
+
+  private static class CapacityBAOSBytesInput extends BytesInput {
+
+    private final CapacityByteArrayOutputStream arrayOut;
+
+    private CapacityBAOSBytesInput(CapacityByteArrayOutputStream arrayOut) {
+      this.arrayOut = arrayOut;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      arrayOut.writeTo(out);
+    }
+
+    @Override
+    public long size() {
+      return arrayOut.size();
+    }
+
+  }
+
+  private static class BAOSBytesInput extends BytesInput {
+
+    private final ByteArrayOutputStream arrayOut;
+
+    private BAOSBytesInput(ByteArrayOutputStream arrayOut) {
+      this.arrayOut = arrayOut;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      arrayOut.writeTo(out);
+    }
+
+    @Override
+    public long size() {
+      return arrayOut.size();
+    }
+
+  }
+
+  private static class ByteArrayBytesInput extends BytesInput {
+
+    private final byte[] in;
+    private final int offset;
+    private final int length;
+
+    private ByteArrayBytesInput(byte[] in, int offset, int length) {
+      this.in = in;
+      this.offset = offset;
+      this.length = length;
+    }
+
+    @Override
+    public void writeAllTo(OutputStream out) throws IOException {
+      out.write(in, offset, length);
+    }
+
+    @Override
+    public long size() {
+      return length;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/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
new file mode 100644
index 0000000..1670f9c
--- /dev/null
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java
@@ -0,0 +1,276 @@
+/* 
+ * 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 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.util.ArrayList;
+import java.util.List;
+
+import org.apache.parquet.Log;
+
+/**
+ * Similar to a {@link ByteArrayOutputStream}, but uses a different strategy for growing that does not involve copying.
+ * Where ByteArrayOutputStream is backed by a single array that "grows" by copying into a new larger array, this output
+ * stream grows by allocating a new array (slab) and adding it to a list of previous arrays.
+ *
+ * Each new slab is allocated to be the same size as all the previous slabs combined, so these allocations become
+ * exponentially less frequent, just like ByteArrayOutputStream, with one difference. This output stream accepts a
+ * max capacity hint, which is a hint describing the max amount of data that will be written to this stream. As the
+ * total size of this stream nears this max, this stream starts to grow linearly instead of exponentially.
+ * So new slabs are allocated to be 1/5th of the max capacity hint,
+ * instead of equal to the total previous size of all slabs. This is useful because it prevents allocating roughly
+ * twice the needed space when a new slab is added just before the stream is done being used.
+ *
+ * When reusing this stream it will adjust the initial slab size based on the previous data size, aiming for fewer
+ * allocations, with the assumption that a similar amount of data will be written to this stream on re-use.
+ * See ({@link CapacityByteArrayOutputStream#reset()}).
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class CapacityByteArrayOutputStream extends OutputStream {
+  private static final Log LOG = Log.getLog(CapacityByteArrayOutputStream.class);
+  private static final byte[] EMPTY_SLAB = new byte[0];
+
+  private int initialSlabSize;
+  private final int maxCapacityHint;
+  private final List<byte[]> slabs = new ArrayList<byte[]>();
+
+  private byte[] currentSlab;
+  private int currentSlabIndex;
+  private int bytesAllocated = 0;
+  private int bytesUsed = 0;
+
+  /**
+   * Return an initial slab size such that a CapacityByteArrayOutputStream constructed with it
+   * will end up allocating targetNumSlabs in order to reach targetCapacity. This aims to be
+   * a balance between the overhead of creating new slabs and wasting memory by eagerly making
+   * initial slabs too big.
+   *
+   * Note that targetCapacity here need not match maxCapacityHint in the constructor of
+   * CapacityByteArrayOutputStream, though often that would make sense.
+   *
+   * @param minSlabSize no matter what we shouldn't make slabs any smaller than this
+   * @param targetCapacity after we've allocated targetNumSlabs how much capacity should we have?
+   * @param targetNumSlabs how many slabs should it take to reach targetCapacity?
+   */
+  public static int initialSlabSizeHeuristic(int minSlabSize, int targetCapacity, int targetNumSlabs) {
+    // initialSlabSize = (targetCapacity / (2^targetNumSlabs)) means we double targetNumSlabs times
+    // before reaching the targetCapacity
+    // eg for page size of 1MB we start at 1024 bytes.
+    // we also don't want to start too small, so we also apply a minimum.
+    return max(minSlabSize, ((int) (targetCapacity / pow(2, targetNumSlabs))));
+  }
+
+  /**
+   * 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) {
+
+    return new CapacityByteArrayOutputStream(
+        initialSlabSizeHeuristic(minSlabSize, maxCapacityHint, targetNumSlabs),
+        maxCapacityHint);
+  }
+
+  /**
+   * Defaults maxCapacityHint to 1MB
+   * @param initialSlabSize
+   * @deprecated use {@link CapacityByteArrayOutputStream#CapacityByteArrayOutputStream(int, int)}
+   */
+  @Deprecated
+  public CapacityByteArrayOutputStream(int initialSlabSize) {
+    this(initialSlabSize, 1024 * 1024);
+  }
+
+  /**
+   * @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) {
+    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;
+    reset();
+  }
+
+  /**
+   * the new slab is guaranteed to be at least minimumSize
+   * @param minimumSize the size of the data we want to copy in the new slab
+   */
+  private void addSlab(int minimumSize) {
+    int nextSlabSize;
+
+    if (bytesUsed == 0) {
+      nextSlabSize = initialSlabSize;
+    } else if (bytesUsed > maxCapacityHint / 5) {
+      // to avoid an overhead of up to twice the needed size, we get linear when approaching target page size
+      nextSlabSize = maxCapacityHint / 5;
+    } else {
+      // double the size every time
+      nextSlabSize = bytesUsed;
+    }
+
+    if (nextSlabSize < minimumSize) {
+      if (Log.DEBUG) LOG.debug(format("slab size %,d too small for value of size %,d. Bumping up slab size", nextSlabSize, minimumSize));
+      nextSlabSize = minimumSize;
+    }
+
+    if (Log.DEBUG) LOG.debug(format("used %d slabs, adding new slab of size %d", slabs.size(), nextSlabSize));
+
+    this.currentSlab = new byte[nextSlabSize];
+    this.slabs.add(currentSlab);
+    this.bytesAllocated += nextSlabSize;
+    this.currentSlabIndex = 0;
+  }
+
+  @Override
+  public void write(int b) {
+    if (currentSlabIndex == currentSlab.length) {
+      addSlab(1);
+    }
+    currentSlab[currentSlabIndex] = (byte) b;
+    currentSlabIndex += 1;
+    bytesUsed += 1;
+  }
+
+  @Override
+  public void write(byte b[], int off, int len) {
+    if ((off < 0) || (off > b.length) || (len < 0) ||
+        ((off + len) - b.length > 0)) {
+      throw new IndexOutOfBoundsException(
+          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);
+      final int length2 = len - length1;
+      addSlab(length2);
+      arraycopy(b, off + length1, currentSlab, currentSlabIndex, length2);
+      currentSlabIndex = length2;
+    } else {
+      arraycopy(b, off, currentSlab, currentSlabIndex, len);
+      currentSlabIndex += len;
+    }
+    bytesUsed += len;
+  }
+
+  /**
+   * Writes the complete contents of this buffer to the specified output stream argument. the output
+   * stream's write method <code>out.write(slab, 0, slab.length)</code>) will be called once per slab.
+   *
+   * @param      out   the output stream to which to write the data.
+   * @exception  IOException  if an I/O error occurs.
+   */
+  public void writeTo(OutputStream out) throws IOException {
+    for (int i = 0; i < slabs.size() - 1; i++) {
+      final byte[] slab = slabs.get(i);
+      out.write(slab);
+    }
+    out.write(currentSlab, 0, currentSlabIndex);
+  }
+
+  /**
+   * @return The total size in bytes of data written to this stream.
+   */
+  public long size() {
+    return bytesUsed;
+  }
+
+  /**
+   *
+   * @return The total size in bytes currently allocated for this stream.
+   */
+  public int getCapacity() {
+    return bytesAllocated;
+  }
+
+  /**
+   * When re-using an instance with reset, it will adjust slab size based on previous data size.
+   * The intent is to reuse the same instance for the same type of data (for example, the same column).
+   * The assumption is that the size in the buffer will be consistent.
+   */
+  public void reset() {
+    // readjust slab size.
+    // 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));
+    this.slabs.clear();
+    this.bytesAllocated = 0;
+    this.bytesUsed = 0;
+    this.currentSlab = EMPTY_SLAB;
+    this.currentSlabIndex = 0;
+  }
+
+  /**
+   * @return the index of the last value written to this stream, which
+   * can be passed to {@link #setByte(long, byte)} in order to change it
+   */
+  public long getCurrentIndex() {
+    checkArgument(bytesUsed > 0, "This is an empty stream");
+    return bytesUsed - 1;
+  }
+
+  /**
+   * Replace the byte stored at position index in this stream with value
+   *
+   * @param index which byte to replace
+   * @param value the value to replace it with
+   */
+  public void setByte(long index, byte value) {
+    checkArgument(index < bytesUsed, "Index: " + index + " is >= the current size of: " + bytesUsed);
+
+    long seen = 0;
+    for (int i = 0; i < slabs.size(); i++) {
+      byte[] slab = slabs.get(i);
+      if (index < seen + slab.length) {
+        // ok found index
+        slab[(int)(index-seen)] = value;
+        break;
+      }
+      seen += slab.length;
+    }
+  }
+
+  /**
+   * @param prefix  a prefix to be used for every new line in the string
+   * @return a text representation of the memory usage of this structure
+   */
+  public String memUsageString(String prefix) {
+    return format("%s %s %d slabs, %,d bytes", prefix, getClass().getSimpleName(), slabs.size(), getCapacity());
+  }
+
+  /**
+   * @return the total number of allocated slabs
+   */
+  int getSlabCount() {
+    return slabs.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-encoding/src/main/java/org/apache/parquet/bytes/ConcatenatingByteArrayCollector.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/main/java/org/apache/parquet/bytes/ConcatenatingByteArrayCollector.java b/parquet-encoding/src/main/java/org/apache/parquet/bytes/ConcatenatingByteArrayCollector.java
new file mode 100644
index 0000000..d333168
--- /dev/null
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/ConcatenatingByteArrayCollector.java
@@ -0,0 +1,63 @@
+/*
+ * 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.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.lang.String.format;
+
+public class ConcatenatingByteArrayCollector extends BytesInput {
+  private final List<byte[]> slabs = new ArrayList<byte[]>();
+  private long size = 0;
+
+  public void collect(BytesInput bytesInput) throws IOException {
+    byte[] bytes = bytesInput.toByteArray();
+    slabs.add(bytes);
+    size += bytes.length;
+  }
+
+  public void reset() {
+    size = 0;
+    slabs.clear();
+  }
+
+  @Override
+  public void writeAllTo(OutputStream out) throws IOException {
+    for (byte[] slab : slabs) {
+      out.write(slab);
+    }
+  }
+
+  @Override
+  public long size() {
+    return size;
+  }
+
+  /**
+   * @param prefix  a prefix to be used for every new line in the string
+   * @return a text representation of the memory usage of this structure
+   */
+  public String memUsageString(String prefix) {
+    return format("%s %s %d slabs, %,d bytes", prefix, getClass().getSimpleName(), slabs.size(), size);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataInputStream.java
----------------------------------------------------------------------
diff --git a/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataInputStream.java b/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataInputStream.java
new file mode 100644
index 0000000..a092753
--- /dev/null
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataInputStream.java
@@ -0,0 +1,424 @@
+/* 
+ * 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.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Based on DataInputStream but little endian and without the String/char methods
+ *
+ * @author Julien Le Dem
+ *
+ */
+public final class LittleEndianDataInputStream extends InputStream {
+
+  private final InputStream in;
+
+  /**
+   * Creates a LittleEndianDataInputStream that uses the specified
+   * underlying InputStream.
+   *
+   * @param  in   the specified input stream
+   */
+  public LittleEndianDataInputStream(InputStream in) {
+    this.in = in;
+  }
+
+  /**
+   * See the general contract of the <code>readFully</code>
+   * method of <code>DataInput</code>.
+   * <p>
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @param      b   the buffer into which the data is read.
+   * @exception  EOFException  if this input stream reaches the end before
+   *             reading all the bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final void readFully(byte b[]) throws IOException {
+    readFully(b, 0, b.length);
+  }
+
+  /**
+   * See the general contract of the <code>readFully</code>
+   * method of <code>DataInput</code>.
+   * <p>
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @param      b     the buffer into which the data is read.
+   * @param      off   the start offset of the data.
+   * @param      len   the number of bytes to read.
+   * @exception  EOFException  if this input stream reaches the end before
+   *               reading all the bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final void readFully(byte b[], int off, int len) throws IOException {
+    if (len < 0)
+      throw new IndexOutOfBoundsException();
+    int n = 0;
+    while (n < len) {
+      int count = in.read(b, off + n, len - n);
+      if (count < 0)
+        throw new EOFException();
+      n += count;
+    }
+  }
+
+  /**
+   * See the general contract of the <code>skipBytes</code>
+   * method of <code>DataInput</code>.
+   * <p>
+   * Bytes for this operation are read from the contained
+   * input stream.
+   *
+   * @param      n   the number of bytes to be skipped.
+   * @return     the actual number of bytes skipped.
+   * @exception  IOException  if the contained input stream does not support
+   *             seek, or the stream has been closed and
+   *             the contained input stream does not support
+   *             reading after close, or another I/O error occurs.
+   */
+  public final int skipBytes(int n) throws IOException {
+    int total = 0;
+    int cur = 0;
+
+    while ((total<n) && ((cur = (int) in.skip(n-total)) > 0)) {
+      total += cur;
+    }
+
+    return total;
+  }
+
+  /**
+   * @return
+   * @throws IOException
+   * @see java.io.InputStream#read()
+   */
+  public int read() throws IOException {
+    return in.read();
+  }
+
+  /**
+   * @return
+   * @see java.lang.Object#hashCode()
+   */
+  public int hashCode() {
+    return in.hashCode();
+  }
+
+  /**
+   * @param b
+   * @return
+   * @throws IOException
+   * @see java.io.InputStream#read(byte[])
+   */
+  public int read(byte[] b) throws IOException {
+    return in.read(b);
+  }
+
+  /**
+   * @param obj
+   * @return
+   * @see java.lang.Object#equals(java.lang.Object)
+   */
+  public boolean equals(Object obj) {
+    return in.equals(obj);
+  }
+
+  /**
+   * @param b
+   * @param off
+   * @param len
+   * @return
+   * @throws IOException
+   * @see java.io.InputStream#read(byte[], int, int)
+   */
+  public int read(byte[] b, int off, int len) throws IOException {
+    return in.read(b, off, len);
+  }
+
+  /**
+   * @param n
+   * @return
+   * @throws IOException
+   * @see java.io.InputStream#skip(long)
+   */
+  public long skip(long n) throws IOException {
+    return in.skip(n);
+  }
+
+  /**
+   * @return
+   * @throws IOException
+   * @see java.io.InputStream#available()
+   */
+  public int available() throws IOException {
+    return in.available();
+  }
+
+  /**
+   * @throws IOException
+   * @see java.io.InputStream#close()
+   */
+  public void close() throws IOException {
+    in.close();
+  }
+
+  /**
+   * @param readlimit
+   * @see java.io.InputStream#mark(int)
+   */
+  public void mark(int readlimit) {
+    in.mark(readlimit);
+  }
+
+  /**
+   * @throws IOException
+   * @see java.io.InputStream#reset()
+   */
+  public void reset() throws IOException {
+    in.reset();
+  }
+
+  /**
+   * @return
+   * @see java.io.InputStream#markSupported()
+   */
+  public boolean markSupported() {
+    return in.markSupported();
+  }
+
+  /**
+   * See the general contract of the <code>readBoolean</code>
+   * method of <code>DataInput</code>.
+   * <p>
+   * Bytes for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the <code>boolean</code> value read.
+   * @exception  EOFException  if this input stream has reached the end.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final boolean readBoolean() throws IOException {
+    int ch = in.read();
+    if (ch < 0)
+      throw new EOFException();
+    return (ch != 0);
+  }
+
+  /**
+   * See the general contract of the <code>readByte</code>
+   * method of <code>DataInput</code>.
+   * <p>
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next byte of this input stream as a signed 8-bit
+   *             <code>byte</code>.
+   * @exception  EOFException  if this input stream has reached the end.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final byte readByte() throws IOException {
+    int ch = in.read();
+    if (ch < 0)
+      throw new EOFException();
+    return (byte)(ch);
+  }
+
+  /**
+   * See the general contract of the <code>readUnsignedByte</code>
+   * method of <code>DataInput</code>.
+   * <p>
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next byte of this input stream, interpreted as an
+   *             unsigned 8-bit number.
+   * @exception  EOFException  if this input stream has reached the end.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see         java.io.FilterInputStream#in
+   */
+  public final int readUnsignedByte() throws IOException {
+    int ch = in.read();
+    if (ch < 0)
+      throw new EOFException();
+    return ch;
+  }
+
+  /**
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next two bytes of this input stream, interpreted as a
+   *             signed 16-bit number.
+   * @exception  EOFException  if this input stream reaches the end before
+   *               reading two bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final short readShort() throws IOException {
+    int ch2 = in.read();
+    int ch1 = in.read();
+    if ((ch1 | ch2) < 0)
+      throw new EOFException();
+    return (short)((ch1 << 8) + (ch2 << 0));
+  }
+
+  /**
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next two bytes of this input stream, interpreted as an
+   *             unsigned 16-bit integer.
+   * @exception  EOFException  if this input stream reaches the end before
+   *             reading two bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final int readUnsignedShort() throws IOException {
+    int ch2 = in.read();
+    int ch1 = in.read();
+    if ((ch1 | ch2) < 0)
+      throw new EOFException();
+    return (ch1 << 8) + (ch2 << 0);
+  }
+
+  /**
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next four bytes of this input stream, interpreted as an
+   *             <code>int</code>.
+   * @exception  EOFException  if this input stream reaches the end before
+   *               reading four bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final int readInt() throws IOException {
+    // TODO: has this been benchmarked against two alternate implementations?
+    // 1) Integer.reverseBytes(in.readInt())
+    // 2) keep a member byte[4], wrapped by an IntBuffer with appropriate endianness set,
+    //    and call IntBuffer.get()
+    // Both seem like they might be faster.
+    int ch4 = in.read();
+    int ch3 = in.read();
+    int ch2 = in.read();
+    int ch1 = in.read();
+    if ((ch1 | ch2 | ch3 | ch4) < 0)
+      throw new EOFException();
+    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
+  }
+
+  private byte readBuffer[] = new byte[8];
+
+  /**
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next eight bytes of this input stream, interpreted as a
+   *             <code>long</code>.
+   * @exception  EOFException  if this input stream reaches the end before
+   *               reading eight bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.io.FilterInputStream#in
+   */
+  public final long readLong() throws IOException {
+    // TODO: see perf question above in readInt
+    readFully(readBuffer, 0, 8);
+    return (((long)readBuffer[7] << 56) +
+        ((long)(readBuffer[6] & 255) << 48) +
+        ((long)(readBuffer[5] & 255) << 40) +
+        ((long)(readBuffer[4] & 255) << 32) +
+        ((long)(readBuffer[3] & 255) << 24) +
+        ((readBuffer[2] & 255) << 16) +
+        ((readBuffer[1] & 255) <<  8) +
+        ((readBuffer[0] & 255) <<  0));
+  }
+
+  /**
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next four bytes of this input stream, interpreted as a
+   *             <code>float</code>.
+   * @exception  EOFException  if this input stream reaches the end before
+   *               reading four bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.lang.Float#intBitsToFloat(int)
+   */
+  public final float readFloat() throws IOException {
+    return Float.intBitsToFloat(readInt());
+  }
+
+  /**
+   * Bytes
+   * for this operation are read from the contained
+   * input stream.
+   *
+   * @return     the next eight bytes of this input stream, interpreted as a
+   *             <code>double</code>.
+   * @exception  EOFException  if this input stream reaches the end before
+   *               reading eight bytes.
+   * @exception  IOException   the stream has been closed and the contained
+   *             input stream does not support reading after close, or
+   *             another I/O error occurs.
+   * @see        java.lang.Double#longBitsToDouble(long)
+   */
+  public final double readDouble() throws IOException {
+    return Double.longBitsToDouble(readLong());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/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
new file mode 100644
index 0000000..da4e92f
--- /dev/null
+++ b/parquet-encoding/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java
@@ -0,0 +1,213 @@
+/* 
+ * 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.OutputStream;
+
+/**
+ * Based on DataOutputStream but in little endian and without the String/char methods
+ *
+ * @author Julien Le Dem
+ *
+ */
+public class LittleEndianDataOutputStream extends OutputStream {
+
+  private final OutputStream out;
+
+  /**
+   * Creates a new data output stream to write data to the specified
+   * underlying output stream. The counter <code>written</code> is
+   * set to zero.
+   *
+   * @param   out   the underlying output stream, to be saved for later
+   *                use.
+   * @see     java.io.FilterOutputStream#out
+   */
+  public LittleEndianDataOutputStream(OutputStream out) {
+    this.out = out;
+  }
+
+  /**
+   * Writes the specified byte (the low eight bits of the argument
+   * <code>b</code>) to the underlying output stream. If no exception
+   * is thrown, the counter <code>written</code> is incremented by
+   * <code>1</code>.
+   * <p>
+   * Implements the <code>write</code> method of <code>OutputStream</code>.
+   *
+   * @param      b   the <code>byte</code> to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   */
+  public void write(int b) throws IOException {
+    out.write(b);
+  }
+
+  /**
+   * Writes <code>len</code> bytes from the specified byte array
+   * starting at offset <code>off</code> to the underlying output stream.
+   * If no exception is thrown, the counter <code>written</code> is
+   * incremented by <code>len</code>.
+   *
+   * @param      b     the data.
+   * @param      off   the start offset in the data.
+   * @param      len   the number of bytes to write.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   */
+  public void write(byte b[], int off, int len) throws IOException {
+    out.write(b, off, len);
+  }
+
+  /**
+   * Flushes this data output stream. This forces any buffered output
+   * bytes to be written out to the stream.
+   * <p>
+   * The <code>flush</code> method of <code>DataOutputStream</code>
+   * calls the <code>flush</code> method of its underlying output stream.
+   *
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   * @see        java.io.OutputStream#flush()
+   */
+  public void flush() throws IOException {
+    out.flush();
+  }
+
+  /**
+   * Writes a <code>boolean</code> to the underlying output stream as
+   * a 1-byte value. The value <code>true</code> is written out as the
+   * value <code>(byte)1</code>; the value <code>false</code> is
+   * written out as the value <code>(byte)0</code>. If no exception is
+   * thrown, the counter <code>written</code> is incremented by
+   * <code>1</code>.
+   *
+   * @param      v   a <code>boolean</code> value to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   */
+  public final void writeBoolean(boolean v) throws IOException {
+    out.write(v ? 1 : 0);
+  }
+
+  /**
+   * Writes out a <code>byte</code> to the underlying output stream as
+   * a 1-byte value. If no exception is thrown, the counter
+   * <code>written</code> is incremented by <code>1</code>.
+   *
+   * @param      v   a <code>byte</code> value to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   */
+  public final void writeByte(int v) throws IOException {
+    out.write(v);
+  }
+
+  /**
+   * Writes a <code>short</code> to the underlying output stream as two
+   * bytes, low byte first. If no exception is thrown, the counter
+   * <code>written</code> is incremented by <code>2</code>.
+   *
+   * @param      v   a <code>short</code> to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   */
+  public final void writeShort(int v) throws IOException {
+    out.write((v >>> 0) & 0xFF);
+    out.write((v >>> 8) & 0xFF);
+  }
+
+  /**
+   * Writes an <code>int</code> to the underlying output stream as four
+   * bytes, low byte first. If no exception is thrown, the counter
+   * <code>written</code> is incremented by <code>4</code>.
+   *
+   * @param      v   an <code>int</code> to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   */
+  public final void writeInt(int v) throws IOException {
+    // TODO: see note in LittleEndianDataInputStream: maybe faster
+    // to use Integer.reverseBytes() and then writeInt, or a ByteBuffer
+    // approach
+    out.write((v >>>  0) & 0xFF);
+    out.write((v >>>  8) & 0xFF);
+    out.write((v >>> 16) & 0xFF);
+    out.write((v >>> 24) & 0xFF);
+  }
+
+  private byte writeBuffer[] = new byte[8];
+
+  /**
+   * Writes a <code>long</code> to the underlying output stream as eight
+   * bytes, low byte first. In no exception is thrown, the counter
+   * <code>written</code> is incremented by <code>8</code>.
+   *
+   * @param      v   a <code>long</code> to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   */
+  public final void writeLong(long v) throws IOException {
+    writeBuffer[7] = (byte)(v >>> 56);
+    writeBuffer[6] = (byte)(v >>> 48);
+    writeBuffer[5] = (byte)(v >>> 40);
+    writeBuffer[4] = (byte)(v >>> 32);
+    writeBuffer[3] = (byte)(v >>> 24);
+    writeBuffer[2] = (byte)(v >>> 16);
+    writeBuffer[1] = (byte)(v >>>  8);
+    writeBuffer[0] = (byte)(v >>>  0);
+    out.write(writeBuffer, 0, 8);
+  }
+
+  /**
+   * Converts the float argument to an <code>int</code> using the
+   * <code>floatToIntBits</code> method in class <code>Float</code>,
+   * and then writes that <code>int</code> value to the underlying
+   * output stream as a 4-byte quantity, low byte first. If no
+   * exception is thrown, the counter <code>written</code> is
+   * incremented by <code>4</code>.
+   *
+   * @param      v   a <code>float</code> value to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   * @see        java.lang.Float#floatToIntBits(float)
+   */
+  public final void writeFloat(float v) throws IOException {
+    writeInt(Float.floatToIntBits(v));
+  }
+
+  /**
+   * Converts the double argument to a <code>long</code> using the
+   * <code>doubleToLongBits</code> method in class <code>Double</code>,
+   * and then writes that <code>long</code> value to the underlying
+   * output stream as an 8-byte quantity, low byte first. If no
+   * exception is thrown, the counter <code>written</code> is
+   * incremented by <code>8</code>.
+   *
+   * @param      v   a <code>double</code> value to be written.
+   * @exception  IOException  if an I/O error occurs.
+   * @see        java.io.FilterOutputStream#out
+   * @see        java.lang.Double#doubleToLongBits(double)
+   */
+  public final void writeDouble(double v) throws IOException {
+    writeLong(Double.doubleToLongBits(v));
+  }
+
+}


Mime
View raw message