Repository: tajo
Updated Branches:
  refs/heads/branch-0.11.0 b2b708024 -> c3c78fc21


http://git-wip-us.apache.org/repos/asf/tajo/blob/c3c78fc2/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/package-info.java
----------------------------------------------------------------------
diff --git 
a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/package-info.java
 
b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/package-info.java
new file mode 100644
index 0000000..ae4841b
--- /dev/null
+++ 
b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/package-info.java
@@ -0,0 +1,37 @@
+/**
+ * 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.
+ */
+/**
+ * <p>
+ * Provides read and write support for ORC files.
+ * Source files in this packages are from Hive.
+ * But, some files are modified for supporting the concept of Tajo Tuple and 
Datum.
+ * Followings are listing of modified files. When updating library, be careful 
for
+ * the files.
+ *
+ *
+ * ColumnStatisticsImpl.java
+ * FileOrcDataSource.java
+ * HdfsOrcDataSource.java
+ * OrcConf.java
+ * OrcFile.java
+ * Writer.java
+ * WriterImpl.java
+ * </p>
+ */
+
+package org.apache.tajo.storage.thirdparty.orc;

http://git-wip-us.apache.org/repos/asf/tajo/blob/c3c78fc2/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/BloomFilter.java
----------------------------------------------------------------------
diff --git 
a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/BloomFilter.java
 
b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/BloomFilter.java
new file mode 100644
index 0000000..2bbd0cd
--- /dev/null
+++ 
b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/BloomFilter.java
@@ -0,0 +1,291 @@
+/**
+ * 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.tajo.storage.thirdparty.orc.util;
+
+import java.util.Arrays;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * BloomFilter is a probabilistic data structure for set membership check. 
BloomFilters are
+ * highly space efficient when compared to using a HashSet. Because of the 
probabilistic nature of
+ * bloom filter false positive (element not present in bloom filter but test() 
says true) are
+ * possible but false negatives are not possible (if element is present then 
test() will never
+ * say false). The false positive probability is configurable (default: 5%) 
depending on which
+ * storage requirement may increase or decrease. Lower the false positive 
probability greater
+ * is the space requirement.
+ * Bloom filters are sensitive to number of elements that will be inserted in 
the bloom filter.
+ * During the creation of bloom filter expected number of entries must be 
specified. If the number
+ * of insertions exceed the specified initial number of entries then false 
positive probability will
+ * increase accordingly.
+ *
+ * Internally, this implementation of bloom filter uses Murmur3 fast 
non-cryptographic hash
+ * algorithm. Although Murmur2 is slightly faster than Murmur3 in Java, it 
suffers from hash
+ * collisions for specific sequence of repeating bytes. Check the following 
link for more info
+ * https://code.google.com/p/smhasher/wiki/MurmurHash2Flaw
+ */
+public class BloomFilter {
+  public static final double DEFAULT_FPP = 0.05;
+  protected BitSet bitSet;
+  protected int numBits;
+  protected int numHashFunctions;
+
+  public BloomFilter() {
+  }
+
+  public BloomFilter(long expectedEntries) {
+    this(expectedEntries, DEFAULT_FPP);
+  }
+
+  public BloomFilter(long expectedEntries, double fpp) {
+    checkArgument(expectedEntries > 0, "expectedEntries should be > 0");
+    checkArgument(fpp > 0.0 && fpp < 1.0, "False positive probability should 
be > 0.0 & < 1.0");
+    int nb = optimalNumOfBits(expectedEntries, fpp);
+    // make 'm' multiple of 64
+    this.numBits = nb + (Long.SIZE - (nb % Long.SIZE));
+    this.numHashFunctions = optimalNumOfHashFunctions(expectedEntries, 
numBits);
+    this.bitSet = new BitSet(numBits);
+  }
+
+  static int optimalNumOfHashFunctions(long n, long m) {
+    return Math.max(1, (int) Math.round((double) m / n * Math.log(2)));
+  }
+
+  static int optimalNumOfBits(long n, double p) {
+    return (int) (-n * Math.log(p) / (Math.log(2) * Math.log(2)));
+  }
+
+  public void add(byte[] val) {
+    if (val == null) {
+      addBytes(val, -1);
+    } else {
+      addBytes(val, val.length);
+    }
+  }
+
+  public void addBytes(byte[] val, int length) {
+    // We use the trick mentioned in "Less Hashing, Same Performance: Building 
a Better Bloom Filter"
+    // by Kirsch et.al. From abstract 'only two hash functions are necessary 
to effectively
+    // implement a Bloom filter without any loss in the asymptotic false 
positive probability'
+
+    // Lets split up 64-bit hashcode into two 32-bit hash codes and employ the 
technique mentioned
+    // in the above paper
+    long hash64 = val == null ? Murmur3.NULL_HASHCODE : Murmur3.hash64(val, 
length);
+    addHash(hash64);
+  }
+
+  private void addHash(long hash64) {
+    int hash1 = (int) hash64;
+    int hash2 = (int) (hash64 >>> 32);
+
+    for (int i = 1; i <= numHashFunctions; i++) {
+      int combinedHash = hash1 + (i * hash2);
+      // hashcode should be positive, flip all the bits if it's negative
+      if (combinedHash < 0) {
+        combinedHash = ~combinedHash;
+      }
+      int pos = combinedHash % numBits;
+      bitSet.set(pos);
+    }
+  }
+
+  public void addString(String val) {
+    if (val == null) {
+      add(null);
+    } else {
+      add(val.getBytes());
+    }
+  }
+
+  public void addLong(long val) {
+    addHash(getLongHash(val));
+  }
+
+  public void addDouble(double val) {
+    addLong(Double.doubleToLongBits(val));
+  }
+
+  public boolean test(byte[] val) {
+    if (val == null) {
+      return testBytes(val, -1);
+    }
+    return testBytes(val, val.length);
+  }
+
+  public boolean testBytes(byte[] val, int length) {
+    long hash64 = val == null ? Murmur3.NULL_HASHCODE : Murmur3.hash64(val, 
length);
+    return testHash(hash64);
+  }
+
+  private boolean testHash(long hash64) {
+    int hash1 = (int) hash64;
+    int hash2 = (int) (hash64 >>> 32);
+
+    for (int i = 1; i <= numHashFunctions; i++) {
+      int combinedHash = hash1 + (i * hash2);
+      // hashcode should be positive, flip all the bits if it's negative
+      if (combinedHash < 0) {
+        combinedHash = ~combinedHash;
+      }
+      int pos = combinedHash % numBits;
+      if (!bitSet.get(pos)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public boolean testString(String val) {
+    if (val == null) {
+      return test(null);
+    } else {
+      return test(val.getBytes());
+    }
+  }
+
+  public boolean testLong(long val) {
+    return testHash(getLongHash(val));
+  }
+
+  // Thomas Wang's integer hash function
+  // 
http://web.archive.org/web/20071223173210/http://www.concentric.net/~Ttwang/tech/inthash.htm
+  private long getLongHash(long key) {
+    key = (~key) + (key << 21); // key = (key << 21) - key - 1;
+    key = key ^ (key >> 24);
+    key = (key + (key << 3)) + (key << 8); // key * 265
+    key = key ^ (key >> 14);
+    key = (key + (key << 2)) + (key << 4); // key * 21
+    key = key ^ (key >> 28);
+    key = key + (key << 31);
+    return key;
+  }
+
+  public boolean testDouble(double val) {
+    return testLong(Double.doubleToLongBits(val));
+  }
+
+  public long sizeInBytes() {
+    return getBitSize() / 8;
+  }
+
+  public int getBitSize() {
+    return bitSet.getData().length * Long.SIZE;
+  }
+
+  public int getNumHashFunctions() {
+    return numHashFunctions;
+  }
+
+  public long[] getBitSet() {
+    return bitSet.getData();
+  }
+
+  @Override
+  public String toString() {
+    return "m: " + numBits + " k: " + numHashFunctions;
+  }
+
+  /**
+   * Merge the specified bloom filter with current bloom filter.
+   *
+   * @param that - bloom filter to merge
+   */
+  public void merge(BloomFilter that) {
+    if (this != that && this.numBits == that.numBits && this.numHashFunctions 
== that.numHashFunctions) {
+      this.bitSet.putAll(that.bitSet);
+    } else {
+      throw new IllegalArgumentException("BloomFilters are not compatible for 
merging." +
+          " this - " + this.toString() + " that - " + that.toString());
+    }
+  }
+
+  public void reset() {
+    this.bitSet.clear();
+  }
+
+  /**
+   * Bare metal bit set implementation. For performance reasons, this 
implementation does not check
+   * for index bounds nor expand the bit set size if the specified index is 
greater than the size.
+   */
+  public class BitSet {
+    private final long[] data;
+
+    public BitSet(long bits) {
+      this(new long[(int) Math.ceil((double) bits / (double) Long.SIZE)]);
+    }
+
+    /**
+     * Deserialize long array as bit set.
+     *
+     * @param data - bit array
+     */
+    public BitSet(long[] data) {
+      assert data.length > 0 : "data length is zero!";
+      this.data = data;
+    }
+
+    /**
+     * Sets the bit at specified index.
+     *
+     * @param index - position
+     */
+    public void set(int index) {
+      data[index >>> 6] |= (1L << index);
+    }
+
+    /**
+     * Returns true if the bit is set in the specified index.
+     *
+     * @param index - position
+     * @return - value at the bit position
+     */
+    public boolean get(int index) {
+      return (data[index >>> 6] & (1L << index)) != 0;
+    }
+
+    /**
+     * Number of bits
+     */
+    public long bitSize() {
+      return (long) data.length * Long.SIZE;
+    }
+
+    public long[] getData() {
+      return data;
+    }
+
+    /**
+     * Combines the two BitArrays using bitwise OR.
+     */
+    public void putAll(BitSet array) {
+      assert data.length == array.data.length :
+          "BitArrays must be of equal length (" + data.length + "!= " + 
array.data.length + ")";
+      for (int i = 0; i < data.length; i++) {
+        data[i] |= array.data[i];
+      }
+    }
+
+    /**
+     * Clear the bit set.
+     */
+    public void clear() {
+      Arrays.fill(data, 0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/c3c78fc2/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/Murmur3.java
----------------------------------------------------------------------
diff --git 
a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/Murmur3.java
 
b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/Murmur3.java
new file mode 100644
index 0000000..98b3ce7
--- /dev/null
+++ 
b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/util/Murmur3.java
@@ -0,0 +1,334 @@
+/**
+ * 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.tajo.storage.thirdparty.orc.util;
+
+/**
+ * Murmur3 is successor to Murmur2 fast non-crytographic hash algorithms.
+ *
+ * Murmur3 32 and 128 bit variants.
+ * 32-bit Java port of 
https://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp#94
+ * 128-bit Java port of 
https://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp#255
+ *
+ * This is a public domain code with no copyrights.
+ * From homepage of MurmurHash (https://code.google.com/p/smhasher/),
+ * "All MurmurHash versions are public domain software, and the author 
disclaims all copyright
+ * to their code."
+ */
+public class Murmur3 {
+  // from 64-bit linear congruential generator
+  public static final long NULL_HASHCODE = 2862933555777941757L;
+
+  // Constants for 32 bit variant
+  private static final int C1_32 = 0xcc9e2d51;
+  private static final int C2_32 = 0x1b873593;
+  private static final int R1_32 = 15;
+  private static final int R2_32 = 13;
+  private static final int M_32 = 5;
+  private static final int N_32 = 0xe6546b64;
+
+  // Constants for 128 bit variant
+  private static final long C1 = 0x87c37b91114253d5L;
+  private static final long C2 = 0x4cf5ad432745937fL;
+  private static final int R1 = 31;
+  private static final int R2 = 27;
+  private static final int R3 = 33;
+  private static final int M = 5;
+  private static final int N1 = 0x52dce729;
+  private static final int N2 = 0x38495ab5;
+
+  private static final int DEFAULT_SEED = 104729;
+
+  /**
+   * Murmur3 32-bit variant.
+   *
+   * @param data - input byte array
+   * @return - hashcode
+   */
+  public static int hash32(byte[] data) {
+    return hash32(data, data.length, DEFAULT_SEED);
+  }
+
+  /**
+   * Murmur3 32-bit variant.
+   *
+   * @param data   - input byte array
+   * @param length - length of array
+   * @param seed   - seed. (default 0)
+   * @return - hashcode
+   */
+  public static int hash32(byte[] data, int length, int seed) {
+    int hash = seed;
+    final int nblocks = length >> 2;
+
+    // body
+    for (int i = 0; i < nblocks; i++) {
+      int i_4 = i << 2;
+      int k = (data[i_4] & 0xff)
+          | ((data[i_4 + 1] & 0xff) << 8)
+          | ((data[i_4 + 2] & 0xff) << 16)
+          | ((data[i_4 + 3] & 0xff) << 24);
+
+      // mix functions
+      k *= C1_32;
+      k = Integer.rotateLeft(k, R1_32);
+      k *= C2_32;
+      hash ^= k;
+      hash = Integer.rotateLeft(hash, R2_32) * M_32 + N_32;
+    }
+
+    // tail
+    int idx = nblocks << 2;
+    int k1 = 0;
+    switch (length - idx) {
+      case 3:
+        k1 ^= data[idx + 2] << 16;
+      case 2:
+        k1 ^= data[idx + 1] << 8;
+      case 1:
+        k1 ^= data[idx];
+
+        // mix functions
+        k1 *= C1_32;
+        k1 = Integer.rotateLeft(k1, R1_32);
+        k1 *= C2_32;
+        hash ^= k1;
+    }
+
+    // finalization
+    hash ^= length;
+    hash ^= (hash >>> 16);
+    hash *= 0x85ebca6b;
+    hash ^= (hash >>> 13);
+    hash *= 0xc2b2ae35;
+    hash ^= (hash >>> 16);
+
+    return hash;
+  }
+
+  /**
+   * Murmur3 64-bit variant. This is essentially MSB 8 bytes of Murmur3 
128-bit variant.
+   *
+   * @param data - input byte array
+   * @return - hashcode
+   */
+  public static long hash64(byte[] data) {
+    return hash64(data, data.length, DEFAULT_SEED);
+  }
+
+  public static long hash64(byte[] data, int length) {
+    return hash64(data, length, DEFAULT_SEED);
+  }
+
+  /**
+   * Murmur3 64-bit variant. This is essentially MSB 8 bytes of Murmur3 
128-bit variant.
+   *
+   * @param data   - input byte array
+   * @param length - length of array
+   * @param seed   - seed. (default is 0)
+   * @return - hashcode
+   */
+  public static long hash64(byte[] data, int length, int seed) {
+    long hash = seed;
+    final int nblocks = length >> 3;
+
+    // body
+    for (int i = 0; i < nblocks; i++) {
+      final int i8 = i << 3;
+      long k = ((long) data[i8] & 0xff)
+          | (((long) data[i8 + 1] & 0xff) << 8)
+          | (((long) data[i8 + 2] & 0xff) << 16)
+          | (((long) data[i8 + 3] & 0xff) << 24)
+          | (((long) data[i8 + 4] & 0xff) << 32)
+          | (((long) data[i8 + 5] & 0xff) << 40)
+          | (((long) data[i8 + 6] & 0xff) << 48)
+          | (((long) data[i8 + 7] & 0xff) << 56);
+
+      // mix functions
+      k *= C1;
+      k = Long.rotateLeft(k, R1);
+      k *= C2;
+      hash ^= k;
+      hash = Long.rotateLeft(hash, R2) * M + N1;
+    }
+
+    // tail
+    long k1 = 0;
+    int tailStart = nblocks << 3;
+    switch (length - tailStart) {
+      case 7:
+        k1 ^= ((long) data[tailStart + 6] & 0xff) << 48;
+      case 6:
+        k1 ^= ((long) data[tailStart + 5] & 0xff) << 40;
+      case 5:
+        k1 ^= ((long) data[tailStart + 4] & 0xff) << 32;
+      case 4:
+        k1 ^= ((long) data[tailStart + 3] & 0xff) << 24;
+      case 3:
+        k1 ^= ((long) data[tailStart + 2] & 0xff) << 16;
+      case 2:
+        k1 ^= ((long) data[tailStart + 1] & 0xff) << 8;
+      case 1:
+        k1 ^= ((long) data[tailStart] & 0xff);
+        k1 *= C1;
+        k1 = Long.rotateLeft(k1, R1);
+        k1 *= C2;
+        hash ^= k1;
+    }
+
+    // finalization
+    hash ^= length;
+    hash = fmix64(hash);
+
+    return hash;
+  }
+
+  /**
+   * Murmur3 128-bit variant.
+   *
+   * @param data - input byte array
+   * @return - hashcode (2 longs)
+   */
+  public static long[] hash128(byte[] data) {
+    return hash128(data, data.length, DEFAULT_SEED);
+  }
+
+  /**
+   * Murmur3 128-bit variant.
+   *
+   * @param data   - input byte array
+   * @param length - length of array
+   * @param seed   - seed. (default is 0)
+   * @return - hashcode (2 longs)
+   */
+  public static long[] hash128(byte[] data, int length, int seed) {
+    long h1 = seed;
+    long h2 = seed;
+    final int nblocks = length >> 4;
+
+    // body
+    for (int i = 0; i < nblocks; i++) {
+      final int i16 = i << 4;
+      long k1 = ((long) data[i16] & 0xff)
+          | (((long) data[i16 + 1] & 0xff) << 8)
+          | (((long) data[i16 + 2] & 0xff) << 16)
+          | (((long) data[i16 + 3] & 0xff) << 24)
+          | (((long) data[i16 + 4] & 0xff) << 32)
+          | (((long) data[i16 + 5] & 0xff) << 40)
+          | (((long) data[i16 + 6] & 0xff) << 48)
+          | (((long) data[i16 + 7] & 0xff) << 56);
+
+      long k2 = ((long) data[i16 + 8] & 0xff)
+          | (((long) data[i16 + 9] & 0xff) << 8)
+          | (((long) data[i16 + 10] & 0xff) << 16)
+          | (((long) data[i16 + 11] & 0xff) << 24)
+          | (((long) data[i16 + 12] & 0xff) << 32)
+          | (((long) data[i16 + 13] & 0xff) << 40)
+          | (((long) data[i16 + 14] & 0xff) << 48)
+          | (((long) data[i16 + 15] & 0xff) << 56);
+
+      // mix functions for k1
+      k1 *= C1;
+      k1 = Long.rotateLeft(k1, R1);
+      k1 *= C2;
+      h1 ^= k1;
+      h1 = Long.rotateLeft(h1, R2);
+      h1 += h2;
+      h1 = h1 * M + N1;
+
+      // mix functions for k2
+      k2 *= C2;
+      k2 = Long.rotateLeft(k2, R3);
+      k2 *= C1;
+      h2 ^= k2;
+      h2 = Long.rotateLeft(h2, R1);
+      h2 += h1;
+      h2 = h2 * M + N2;
+    }
+
+    // tail
+    long k1 = 0;
+    long k2 = 0;
+    int tailStart = nblocks << 4;
+    switch (length - tailStart) {
+      case 15:
+        k2 ^= (long) (data[tailStart + 14] & 0xff) << 48;
+      case 14:
+        k2 ^= (long) (data[tailStart + 13] & 0xff) << 40;
+      case 13:
+        k2 ^= (long) (data[tailStart + 12] & 0xff) << 32;
+      case 12:
+        k2 ^= (long) (data[tailStart + 11] & 0xff) << 24;
+      case 11:
+        k2 ^= (long) (data[tailStart + 10] & 0xff) << 16;
+      case 10:
+        k2 ^= (long) (data[tailStart + 9] & 0xff) << 8;
+      case 9:
+        k2 ^= (long) (data[tailStart + 8] & 0xff);
+        k2 *= C2;
+        k2 = Long.rotateLeft(k2, R3);
+        k2 *= C1;
+        h2 ^= k2;
+
+      case 8:
+        k1 ^= (long) (data[tailStart + 7] & 0xff) << 56;
+      case 7:
+        k1 ^= (long) (data[tailStart + 6] & 0xff) << 48;
+      case 6:
+        k1 ^= (long) (data[tailStart + 5] & 0xff) << 40;
+      case 5:
+        k1 ^= (long) (data[tailStart + 4] & 0xff) << 32;
+      case 4:
+        k1 ^= (long) (data[tailStart + 3] & 0xff) << 24;
+      case 3:
+        k1 ^= (long) (data[tailStart + 2] & 0xff) << 16;
+      case 2:
+        k1 ^= (long) (data[tailStart + 1] & 0xff) << 8;
+      case 1:
+        k1 ^= (long) (data[tailStart] & 0xff);
+        k1 *= C1;
+        k1 = Long.rotateLeft(k1, R1);
+        k1 *= C2;
+        h1 ^= k1;
+    }
+
+    // finalization
+    h1 ^= length;
+    h2 ^= length;
+
+    h1 += h2;
+    h2 += h1;
+
+    h1 = fmix64(h1);
+    h2 = fmix64(h2);
+
+    h1 += h2;
+    h2 += h1;
+
+    return new long[]{h1, h2};
+  }
+
+  private static long fmix64(long h) {
+    h ^= (h >>> 33);
+    h *= 0xff51afd7ed558ccdL;
+    h ^= (h >>> 33);
+    h *= 0xc4ceb9fe1a85ec53L;
+    h ^= (h >>> 33);
+    return h;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/c3c78fc2/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto 
b/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto
new file mode 100644
index 0000000..c80cf6c
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto
@@ -0,0 +1,217 @@
+/**
+ * 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.tajo.storage.thirdparty.orc;
+
+message IntegerStatistics  {
+  optional sint64 minimum = 1;
+  optional sint64 maximum = 2;
+  optional sint64 sum = 3;
+}
+
+message DoubleStatistics {
+  optional double minimum = 1;
+  optional double maximum = 2;
+  optional double sum = 3;
+}
+
+message StringStatistics {
+  optional string minimum = 1;
+  optional string maximum = 2;
+  // sum will store the total length of all strings in a stripe
+  optional sint64 sum = 3;
+}
+
+message BucketStatistics {
+  repeated uint64 count = 1 [packed=true];
+}
+
+message DecimalStatistics {
+  optional string minimum = 1;
+  optional string maximum = 2;
+  optional string sum = 3;
+}
+
+message DateStatistics {
+  // min,max values saved as days since epoch
+  optional sint32 minimum = 1;
+  optional sint32 maximum = 2;
+}
+
+message TimestampStatistics {
+  // min,max values saved as milliseconds since epoch
+  optional sint64 minimum = 1;
+  optional sint64 maximum = 2;
+}
+
+message BinaryStatistics {
+  // sum will store the total binary blob length in a stripe
+  optional sint64 sum = 1;
+}
+
+message ColumnStatistics {
+  optional uint64 numberOfValues = 1;
+  optional IntegerStatistics intStatistics = 2;
+  optional DoubleStatistics doubleStatistics = 3;
+  optional StringStatistics stringStatistics = 4;
+  optional BucketStatistics bucketStatistics = 5;
+  optional DecimalStatistics decimalStatistics = 6;
+  optional DateStatistics dateStatistics = 7;
+  optional BinaryStatistics binaryStatistics = 8;
+  optional TimestampStatistics timestampStatistics = 9;
+  optional bool hasNull = 10;
+}
+
+message RowIndexEntry {
+  repeated uint64 positions = 1 [packed=true];
+  optional ColumnStatistics statistics = 2;
+}
+
+message RowIndex {
+  repeated RowIndexEntry entry = 1;
+}
+
+message BloomFilter {
+  optional uint32 numHashFunctions = 1;
+  repeated fixed64 bitset = 2;
+}
+
+message BloomFilterIndex {
+  repeated BloomFilter bloomFilter = 1;
+}
+
+message Stream {
+  // if you add new index stream kinds, you need to make sure to update
+  // StreamName to ensure it is added to the stripe in the right area
+  enum Kind {
+    PRESENT = 0;
+    DATA = 1;
+    LENGTH = 2;
+    DICTIONARY_DATA = 3;
+    DICTIONARY_COUNT = 4;
+    SECONDARY = 5;
+    ROW_INDEX = 6;
+    BLOOM_FILTER = 7;
+  }
+  optional Kind kind = 1;
+  optional uint32 column = 2;
+  optional uint64 length = 3;
+}
+
+message ColumnEncoding {
+  enum Kind {
+    DIRECT = 0;
+    DICTIONARY = 1;
+    DIRECT_V2 = 2;
+    DICTIONARY_V2 = 3;
+  }
+  optional Kind kind = 1;
+  optional uint32 dictionarySize = 2;
+}
+
+message StripeFooter {
+  repeated Stream streams = 1;
+  repeated ColumnEncoding columns = 2;
+  optional string writerTimezone = 3;
+}
+
+message Type {
+  enum Kind {
+    BOOLEAN = 0;
+    BYTE = 1;
+    SHORT = 2;
+    INT = 3;
+    LONG = 4;
+    FLOAT = 5;
+    DOUBLE = 6;
+    STRING = 7;
+    BINARY = 8;
+    TIMESTAMP = 9;
+    LIST = 10;
+    MAP = 11;
+    STRUCT = 12;
+    UNION = 13;
+    DECIMAL = 14;
+    DATE = 15;
+    VARCHAR = 16;
+    CHAR = 17;
+  }
+  optional Kind kind = 1;
+  repeated uint32 subtypes = 2 [packed=true];
+  repeated string fieldNames = 3;
+  optional uint32 maximumLength = 4;
+  optional uint32 precision = 5;
+  optional uint32 scale = 6;
+}
+
+message StripeInformation {
+  optional uint64 offset = 1;
+  optional uint64 indexLength = 2;
+  optional uint64 dataLength = 3;
+  optional uint64 footerLength = 4;
+  optional uint64 numberOfRows = 5;
+}
+
+message UserMetadataItem {
+  optional string name = 1;
+  optional bytes value = 2;
+}
+
+message StripeStatistics {
+  repeated ColumnStatistics colStats = 1;
+}
+
+message Metadata {
+  repeated StripeStatistics stripeStats = 1;
+}
+
+message Footer {
+  optional uint64 headerLength = 1;
+  optional uint64 contentLength = 2;
+  repeated StripeInformation stripes = 3;
+  repeated Type types = 4;
+  repeated UserMetadataItem metadata = 5;
+  optional uint64 numberOfRows = 6;
+  repeated ColumnStatistics statistics = 7;
+  optional uint32 rowIndexStride = 8;
+}
+
+enum CompressionKind {
+  NONE = 0;
+  ZLIB = 1;
+  SNAPPY = 2;
+  LZO = 3;
+}
+
+// Serialized length must be less that 255 bytes
+message PostScript {
+  optional uint64 footerLength = 1;
+  optional CompressionKind compression = 2;
+  optional uint64 compressionBlockSize = 3;
+  // the version of the file format
+  //   [0, 11] = Hive 0.11
+  //   [0, 12] = Hive 0.12
+  repeated uint32 version = 4 [packed = true];
+  optional uint64 metadataLength = 5;
+  // Version of the writer:
+  //   0 (or missing) = original
+  //   1 = HIVE-8732 fixed
+  optional uint32 writerVersion = 6;
+  // Leave this last in the record
+  optional string magic = 8000;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/c3c78fc2/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git 
a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
 
b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
index 66d86f2..02472eb 100644
--- 
a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ 
b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -61,7 +61,6 @@ import static org.junit.Assert.assertTrue;
 @RunWith(Parameterized.class)
 public class TestStorages {
   private TajoConf conf;
-  private static String TEST_PATH = "target/test-data/TestStorages";
 
   private static String TEST_PROJECTION_AVRO_SCHEMA =
       "{\n" +
@@ -120,6 +119,8 @@ public class TestStorages {
 
   public TestStorages(String type, boolean splitable, boolean statsable, 
boolean seekable, boolean internalType)
       throws IOException {
+    final String TEST_PATH = "target/test-data/TestStorages";
+
     this.storeType = type;
     this.splitable = splitable;
     this.statsable = statsable;
@@ -143,6 +144,7 @@ public class TestStorages {
         {BuiltinStorages.DRAW, false, true, true, true},
         {BuiltinStorages.RCFILE, true, true, false, false},
         {BuiltinStorages.PARQUET, false, false, false, false},
+        {BuiltinStorages.ORC, false, true, false, false},
         {BuiltinStorages.SEQUENCE_FILE, true, true, false, false},
         {BuiltinStorages.AVRO, false, false, false, false},
         {BuiltinStorages.TEXT, true, true, true, false},
@@ -1005,7 +1007,8 @@ public class TestStorages {
   public void testLessThanSchemaSize() throws IOException {
     /* Internal storage must be same with schema size */
     if (internalType || storeType.equalsIgnoreCase(BuiltinStorages.AVRO)
-        || storeType.equalsIgnoreCase(BuiltinStorages.PARQUET)) {
+        || storeType.equalsIgnoreCase(BuiltinStorages.PARQUET)
+        || storeType.equalsIgnoreCase(BuiltinStorages.ORC)) {
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/c3c78fc2/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/orc/TestORCScanner.java
----------------------------------------------------------------------
diff --git 
a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/orc/TestORCScanner.java
 
b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/orc/TestORCScanner.java
deleted file mode 100644
index b411793..0000000
--- 
a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/orc/TestORCScanner.java
+++ /dev/null
@@ -1,107 +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 org.apache.tajo.storage.orc;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.TimestampDatum;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.KeyValueSet;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.net.URL;
-
-public class TestORCScanner {
-  private ORCScanner orcScanner;
-
-  public static Path getResourcePath(String path, String suffix) {
-    URL resultBaseURL = ClassLoader.getSystemResource(path);
-    return new Path(resultBaseURL.toString(), suffix);
-  }
-
-  private static FileFragment getFileFragment(Configuration conf, String 
fileName) throws IOException {
-    Path tablePath = new Path(getResourcePath("dataset", "."), fileName);
-    FileSystem fs = FileSystem.getLocal(conf);
-    FileStatus status = fs.getFileStatus(tablePath);
-    return new FileFragment("table", tablePath, 0, status.getLen());
-  }
-
-  @Before
-  public void setup() throws IOException {
-    Schema schema = new Schema();
-    schema.addColumn("userid", TajoDataTypes.Type.INT4);
-    schema.addColumn("movieid", TajoDataTypes.Type.INT4);
-    schema.addColumn("rating", TajoDataTypes.Type.INT2);
-    schema.addColumn("unixtimestamp", TajoDataTypes.Type.TEXT);
-    schema.addColumn("faketime", TajoDataTypes.Type.TIMESTAMP);
-
-    Configuration conf = new TajoConf();
-
-    TableMeta meta = new TableMeta("ORC", new KeyValueSet());
-
-    Fragment fragment = getFileFragment(conf, "u_data_20.orc");
-
-    orcScanner = new ORCScanner(conf, schema, meta, fragment);
-
-    orcScanner.init();
-  }
-
-  @Test
-  public void testReadTuple() {
-    try {
-      Tuple tuple = orcScanner.next();
-
-      assertEquals(tuple.getInt4(0), 196);
-      assertEquals(tuple.getInt4(1), 242);
-      assertEquals(tuple.getInt2(2), 3);
-      assertEquals(tuple.getText(3), "881250949");
-
-      // Timestamp test
-      TimestampDatum timestamp = (TimestampDatum)tuple.asDatum(4);
-
-      assertEquals(timestamp.getYear(), 2008);
-      assertEquals(timestamp.getMonthOfYear(), 12);
-      assertEquals(timestamp.getDayOfMonth(), 12);
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
-  @After
-  public void end() {
-    try {
-      orcScanner.close();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/c3c78fc2/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git 
a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml 
b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
index b5a60fe..6f7e53b 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
@@ -133,7 +133,7 @@
   <!--- Appender Handler -->
   <property>
     <name>tajo.storage.appender-handler</name>
-    <value>text,raw,draw,rcfile,row,parquet,sequencefile,avro</value>
+    <value>text,raw,draw,rcfile,row,parquet,orc,sequencefile,avro</value>
   </property>
 
   <property>
@@ -172,6 +172,11 @@
   </property>
 
   <property>
+    <name>tajo.storage.appender-handler.orc.class</name>
+    <value>org.apache.tajo.storage.orc.ORCAppender</value>
+  </property>
+
+  <property>
     <name>tajo.storage.appender-handler.sequencefile.class</name>
     <value>org.apache.tajo.storage.sequencefile.SequenceFileAppender</value>
   </property>

Reply via email to