hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From the...@apache.org
Subject [1/2] hive git commit: HIVE-10382 : Aggregate stats cache for RDBMS based metastore codepath (Vaibhav Gumashta via Thejas Nair)
Date Tue, 28 Apr 2015 18:06:33 GMT
Repository: hive
Updated Branches:
  refs/heads/branch-1.2 904f01b7d -> 49f13bc74


http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilter.java
deleted file mode 100644
index 6ab0270..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilter.java
+++ /dev/null
@@ -1,298 +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.hadoop.hive.ql.io.filters;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.io.orc.OrcProto;
-
-import com.google.common.primitives.Longs;
-
-/**
- * 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;
-  private BitSet bitSet;
-  private int m;
-  private int k;
-
-  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.m = nb + (Long.SIZE - (nb % Long.SIZE));
-    this.k = optimalNumOfHashFunctions(expectedEntries, m);
-    this.bitSet = new BitSet(m);
-  }
-
-  public BloomFilter(OrcProto.BloomFilter bloomFilter) {
-    this.bitSet = new BitSet(Longs.toArray(bloomFilter.getBitsetList()));
-    this.k = bloomFilter.getNumHashFunctions();
-    this.m = (int) this.bitSet.bitSize();
-  }
-
-  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 <= k; 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 % m;
-      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 <= k; 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 % m;
-      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 k;
-  }
-
-  public long[] getBitSet() {
-    return bitSet.getData();
-  }
-
-  @Override
-  public String toString() {
-    return "m: " + m + " k: " + k;
-  }
-
-  /**
-   * Merge the specified bloom filter with current bloom filter.
-   *
-   * @param that - bloom filter to merge
-   */
-  public void merge(BloomFilter that) {
-    if (this != that && this.m == that.m && this.k == that.k) {
-      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.
-   */
-  private class BitSet {
-    final long[] data;
-
-    BitSet(long bits) {
-      this(new long[(int) Math.ceil((double) bits / (double) Long.SIZE)]);
-    }
-
-    /**
-     * Deserialize long array as bit set.
-     *
-     * @param data - bit array
-     */
-    BitSet(long[] data) {
-      assert data.length > 0 : "data length is zero!";
-      this.data = data;
-    }
-
-    /**
-     * Sets the bit at specified index.
-     *
-     * @param index - position
-     */
-    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
-     */
-    boolean get(int index) {
-      return (data[index >>> 6] & (1L << index)) != 0;
-    }
-
-    /**
-     * Number of bits
-     */
-    long bitSize() {
-      return (long) data.length * Long.SIZE;
-    }
-
-    long[] getData() {
-      return data;
-    }
-
-    /**
-     * Combines the two BitArrays using bitwise OR.
-     */
-    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/hive/blob/49f13bc7/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilterIO.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilterIO.java b/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilterIO.java
new file mode 100644
index 0000000..56aec9f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/filters/BloomFilterIO.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io.filters;
+
+import org.apache.hadoop.hive.ql.io.orc.OrcProto;
+import org.apache.hive.common.util.BloomFilter;
+
+import com.google.common.primitives.Longs;
+
+public class BloomFilterIO extends BloomFilter {
+
+  public BloomFilterIO(long expectedEntries) {
+    super(expectedEntries, DEFAULT_FPP);
+  }
+
+  public BloomFilterIO(long expectedEntries, double fpp) {
+    super(expectedEntries, fpp);
+  }
+
+/**
+ * Initializes the BloomFilter from the given Orc BloomFilter
+ */
+  public BloomFilterIO(OrcProto.BloomFilter bloomFilter) {
+    this.bitSet = new BitSet(Longs.toArray(bloomFilter.getBitsetList()));
+    this.numHashFunctions = bloomFilter.getNumHashFunctions();
+    this.numBits = (int) this.bitSet.bitSize();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/java/org/apache/hadoop/hive/ql/io/filters/Murmur3.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/filters/Murmur3.java b/ql/src/java/org/apache/hadoop/hive/ql/io/filters/Murmur3.java
deleted file mode 100644
index e733892..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/filters/Murmur3.java
+++ /dev/null
@@ -1,334 +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.hadoop.hive.ql.io.filters;
-
-/**
- * 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/hive/blob/49f13bc7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
index 7bfd781..cd4db75 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
@@ -33,7 +33,7 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.ql.io.filters.BloomFilter;
+import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.ql.io.orc.OrcProto.RowIndex;
 import org.apache.hadoop.hive.ql.io.orc.OrcProto.RowIndexEntry;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
@@ -197,12 +197,12 @@ public final class FileDump {
   private static String getFormattedBloomFilters(int col,
       OrcProto.BloomFilterIndex[] bloomFilterIndex) {
     StringBuilder buf = new StringBuilder();
-    BloomFilter stripeLevelBF = null;
+    BloomFilterIO stripeLevelBF = null;
     if (bloomFilterIndex != null && bloomFilterIndex[col] != null) {
       int idx = 0;
       buf.append("\n    Bloom filters for column ").append(col).append(":");
       for (OrcProto.BloomFilter bf : bloomFilterIndex[col].getBloomFilterList()) {
-        BloomFilter toMerge = new BloomFilter(bf);
+        BloomFilterIO toMerge = new BloomFilterIO(bf);
         buf.append("\n      Entry ").append(idx++).append(":").append(getBloomFilterStats(toMerge));
         if (stripeLevelBF == null) {
           stripeLevelBF = toMerge;
@@ -216,7 +216,7 @@ public final class FileDump {
     return buf.toString();
   }
 
-  private static String getBloomFilterStats(BloomFilter bf) {
+  private static String getBloomFilterStats(BloomFilterIO bf) {
     StringBuilder sb = new StringBuilder();
     int bitCount = bf.getBitSize();
     int popCount = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
index 49a8e80..61ee8b9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.io.filters.BloomFilter;
+import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 /**
@@ -301,7 +301,7 @@ public final class OrcFile {
 
       paddingTolerance = conf.getFloat(HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.varname,
           HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.defaultFloatVal);
-      bloomFilterFpp = BloomFilter.DEFAULT_FPP;
+      bloomFilterFpp = BloomFilterIO.DEFAULT_FPP;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index bde9fc2..9e7ac4b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.io.filters.BloomFilter;
+import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.ql.io.orc.RecordReaderUtils.ByteBufferAllocatorPool;
 import org.apache.hadoop.hive.ql.io.orc.TreeReaderFactory.TreeReader;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
@@ -333,9 +333,9 @@ class RecordReaderImpl implements RecordReader {
     ColumnStatistics cs = ColumnStatisticsImpl.deserialize(statsProto);
     Object minValue = getMin(cs);
     Object maxValue = getMax(cs);
-    BloomFilter bf = null;
+    BloomFilterIO bf = null;
     if (bloomFilter != null) {
-      bf = new BloomFilter(bloomFilter);
+      bf = new BloomFilterIO(bloomFilter);
     }
     return evaluatePredicateRange(predicate, minValue, maxValue, cs.hasNull(), bf);
   }
@@ -349,14 +349,14 @@ class RecordReaderImpl implements RecordReader {
    *   predicate.
    */
   static TruthValue evaluatePredicate(ColumnStatistics stats,
-      PredicateLeaf predicate, BloomFilter bloomFilter) {
+      PredicateLeaf predicate, BloomFilterIO bloomFilter) {
     Object minValue = getMin(stats);
     Object maxValue = getMax(stats);
     return evaluatePredicateRange(predicate, minValue, maxValue, stats.hasNull(), bloomFilter);
   }
 
   static TruthValue evaluatePredicateRange(PredicateLeaf predicate, Object min,
-      Object max, boolean hasNull, BloomFilter bloomFilter) {
+      Object max, boolean hasNull, BloomFilterIO bloomFilter) {
     // if we didn't have any values, everything must have been null
     if (min == null) {
       if (predicate.getOperator() == PredicateLeaf.Operator.IS_NULL) {
@@ -490,7 +490,7 @@ class RecordReaderImpl implements RecordReader {
   }
 
   private static TruthValue evaluatePredicateBloomFilter(PredicateLeaf predicate, Object predObj,
-      BloomFilter bloomFilter, boolean hasNull) {
+      BloomFilterIO bloomFilter, boolean hasNull) {
     switch (predicate.getOperator()) {
       case NULL_SAFE_EQUALS:
         // null safe equals does not return *_NULL variant. So set hasNull to false
@@ -511,7 +511,7 @@ class RecordReaderImpl implements RecordReader {
     }
   }
 
-  private static TruthValue checkInBloomFilter(BloomFilter bf, Object predObj, boolean hasNull) {
+  private static TruthValue checkInBloomFilter(BloomFilterIO bf, Object predObj, boolean hasNull) {
     TruthValue result = hasNull ? TruthValue.NO_NULL : TruthValue.NO;
 
     if (predObj instanceof Long) {

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
index fb7aa81..c8b0ffc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.IOConstants;
-import org.apache.hadoop.hive.ql.io.filters.BloomFilter;
+import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.ql.io.orc.CompressionCodec.Modifier;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile.CompressionStrategy;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile.EncodingStrategy;
@@ -638,7 +638,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final OrcProto.RowIndexEntry.Builder rowIndexEntry;
     private final PositionedOutputStream rowIndexStream;
     private final PositionedOutputStream bloomFilterStream;
-    protected final BloomFilter bloomFilter;
+    protected final BloomFilterIO bloomFilter;
     protected final boolean createBloomFilter;
     private final OrcProto.BloomFilterIndex.Builder bloomFilterIndex;
     private final OrcProto.BloomFilter.Builder bloomFilterEntry;
@@ -686,7 +686,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
         bloomFilterEntry = OrcProto.BloomFilter.newBuilder();
         bloomFilterIndex = OrcProto.BloomFilterIndex.newBuilder();
         bloomFilterStream = streamFactory.createStream(id, OrcProto.Stream.Kind.BLOOM_FILTER);
-        bloomFilter = new BloomFilter(streamFactory.getRowIndexStride(),
+        bloomFilter = new BloomFilterIO(streamFactory.getRowIndexStride(),
             streamFactory.getBloomFilterFPP());
       } else {
         bloomFilterEntry = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestBloomFilter.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestBloomFilter.java b/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestBloomFilter.java
deleted file mode 100644
index 32b95ab..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestBloomFilter.java
+++ /dev/null
@@ -1,458 +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.hadoop.hive.ql.io.filters;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Random;
-
-import org.junit.Test;
-
-/**
- *
- */
-public class TestBloomFilter {
-  private static final int COUNT = 100;
-  Random rand = new Random(123);
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testBloomIllegalArg1() {
-    BloomFilter bf = new BloomFilter(0, 0);
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testBloomIllegalArg2() {
-    BloomFilter bf = new BloomFilter(0, 0.1);
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testBloomIllegalArg3() {
-    BloomFilter bf = new BloomFilter(1, 0.0);
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testBloomIllegalArg4() {
-    BloomFilter bf = new BloomFilter(1, 1.0);
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testBloomIllegalArg5() {
-    BloomFilter bf = new BloomFilter(-1, -1);
-  }
-
-
-  @Test
-  public void testBloomNumBits() {
-    assertEquals(0, BloomFilter.optimalNumOfBits(0, 0));
-    assertEquals(0, BloomFilter.optimalNumOfBits(0, 1));
-    assertEquals(0, BloomFilter.optimalNumOfBits(1, 1));
-    assertEquals(7, BloomFilter.optimalNumOfBits(1, 0.03));
-    assertEquals(72, BloomFilter.optimalNumOfBits(10, 0.03));
-    assertEquals(729, BloomFilter.optimalNumOfBits(100, 0.03));
-    assertEquals(7298, BloomFilter.optimalNumOfBits(1000, 0.03));
-    assertEquals(72984, BloomFilter.optimalNumOfBits(10000, 0.03));
-    assertEquals(729844, BloomFilter.optimalNumOfBits(100000, 0.03));
-    assertEquals(7298440, BloomFilter.optimalNumOfBits(1000000, 0.03));
-    assertEquals(6235224, BloomFilter.optimalNumOfBits(1000000, 0.05));
-  }
-
-  @Test
-  public void testBloomNumHashFunctions() {
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(-1, -1));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(0, 0));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(10, 0));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(10, 10));
-    assertEquals(7, BloomFilter.optimalNumOfHashFunctions(10, 100));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(100, 100));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(1000, 100));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(10000, 100));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(100000, 100));
-    assertEquals(1, BloomFilter.optimalNumOfHashFunctions(1000000, 100));
-  }
-
-  @Test
-  public void testBloomFilterBytes() {
-    BloomFilter bf = new BloomFilter(10000);
-    byte[] val = new byte[]{1, 2, 3};
-    byte[] val1 = new byte[]{1, 2, 3, 4};
-    byte[] val2 = new byte[]{1, 2, 3, 4, 5};
-    byte[] val3 = new byte[]{1, 2, 3, 4, 5, 6};
-
-    assertEquals(false, bf.test(val));
-    assertEquals(false, bf.test(val1));
-    assertEquals(false, bf.test(val2));
-    assertEquals(false, bf.test(val3));
-    bf.add(val);
-    assertEquals(true, bf.test(val));
-    assertEquals(false, bf.test(val1));
-    assertEquals(false, bf.test(val2));
-    assertEquals(false, bf.test(val3));
-    bf.add(val1);
-    assertEquals(true, bf.test(val));
-    assertEquals(true, bf.test(val1));
-    assertEquals(false, bf.test(val2));
-    assertEquals(false, bf.test(val3));
-    bf.add(val2);
-    assertEquals(true, bf.test(val));
-    assertEquals(true, bf.test(val1));
-    assertEquals(true, bf.test(val2));
-    assertEquals(false, bf.test(val3));
-    bf.add(val3);
-    assertEquals(true, bf.test(val));
-    assertEquals(true, bf.test(val1));
-    assertEquals(true, bf.test(val2));
-    assertEquals(true, bf.test(val3));
-
-    byte[] randVal = new byte[COUNT];
-    for (int i = 0; i < COUNT; i++) {
-      rand.nextBytes(randVal);
-      bf.add(randVal);
-    }
-    // last value should be present
-    assertEquals(true, bf.test(randVal));
-    // most likely this value should not exist
-    randVal[0] = 0;
-    randVal[1] = 0;
-    randVal[2] = 0;
-    randVal[3] = 0;
-    randVal[4] = 0;
-    assertEquals(false, bf.test(randVal));
-
-    assertEquals(7800, bf.sizeInBytes());
-  }
-
-  @Test
-  public void testBloomFilterByte() {
-    BloomFilter bf = new BloomFilter(10000);
-    byte val = Byte.MIN_VALUE;
-    byte val1 = 1;
-    byte val2 = 2;
-    byte val3 = Byte.MAX_VALUE;
-
-    assertEquals(false, bf.testLong(val));
-    assertEquals(false, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(false, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val1);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val2);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(true, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val3);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(true, bf.testLong(val2));
-    assertEquals(true, bf.testLong(val3));
-
-    byte randVal = 0;
-    for (int i = 0; i < COUNT; i++) {
-      randVal = (byte) rand.nextInt(Byte.MAX_VALUE);
-      bf.addLong(randVal);
-    }
-    // last value should be present
-    assertEquals(true, bf.testLong(randVal));
-    // most likely this value should not exist
-    assertEquals(false, bf.testLong((byte) -120));
-
-    assertEquals(7800, bf.sizeInBytes());
-  }
-
-  @Test
-  public void testBloomFilterInt() {
-    BloomFilter bf = new BloomFilter(10000);
-    int val = Integer.MIN_VALUE;
-    int val1 = 1;
-    int val2 = 2;
-    int val3 = Integer.MAX_VALUE;
-
-    assertEquals(false, bf.testLong(val));
-    assertEquals(false, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(false, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val1);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val2);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(true, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val3);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(true, bf.testLong(val2));
-    assertEquals(true, bf.testLong(val3));
-
-    int randVal = 0;
-    for (int i = 0; i < COUNT; i++) {
-      randVal = rand.nextInt();
-      bf.addLong(randVal);
-    }
-    // last value should be present
-    assertEquals(true, bf.testLong(randVal));
-    // most likely this value should not exist
-    assertEquals(false, bf.testLong(-120));
-
-    assertEquals(7800, bf.sizeInBytes());
-  }
-
-  @Test
-  public void testBloomFilterLong() {
-    BloomFilter bf = new BloomFilter(10000);
-    long val = Long.MIN_VALUE;
-    long val1 = 1;
-    long val2 = 2;
-    long val3 = Long.MAX_VALUE;
-
-    assertEquals(false, bf.testLong(val));
-    assertEquals(false, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(false, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val1);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(false, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val2);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(true, bf.testLong(val2));
-    assertEquals(false, bf.testLong(val3));
-    bf.addLong(val3);
-    assertEquals(true, bf.testLong(val));
-    assertEquals(true, bf.testLong(val1));
-    assertEquals(true, bf.testLong(val2));
-    assertEquals(true, bf.testLong(val3));
-
-    long randVal = 0;
-    for (int i = 0; i < COUNT; i++) {
-      randVal = rand.nextLong();
-      bf.addLong(randVal);
-    }
-    // last value should be present
-    assertEquals(true, bf.testLong(randVal));
-    // most likely this value should not exist
-    assertEquals(false, bf.testLong(-120));
-
-    assertEquals(7800, bf.sizeInBytes());
-  }
-
-  @Test
-  public void testBloomFilterFloat() {
-    BloomFilter bf = new BloomFilter(10000);
-    float val = Float.MIN_VALUE;
-    float val1 = 1.1f;
-    float val2 = 2.2f;
-    float val3 = Float.MAX_VALUE;
-
-    assertEquals(false, bf.testDouble(val));
-    assertEquals(false, bf.testDouble(val1));
-    assertEquals(false, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(false, bf.testDouble(val1));
-    assertEquals(false, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val1);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(true, bf.testDouble(val1));
-    assertEquals(false, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val2);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(true, bf.testDouble(val1));
-    assertEquals(true, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val3);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(true, bf.testDouble(val1));
-    assertEquals(true, bf.testDouble(val2));
-    assertEquals(true, bf.testDouble(val3));
-
-    float randVal = 0;
-    for (int i = 0; i < COUNT; i++) {
-      randVal = rand.nextFloat();
-      bf.addDouble(randVal);
-    }
-    // last value should be present
-    assertEquals(true, bf.testDouble(randVal));
-    // most likely this value should not exist
-    assertEquals(false, bf.testDouble(-120.2f));
-
-    assertEquals(7800, bf.sizeInBytes());
-  }
-
-  @Test
-  public void testBloomFilterDouble() {
-    BloomFilter bf = new BloomFilter(10000);
-    double val = Double.MIN_VALUE;
-    double val1 = 1.1d;
-    double val2 = 2.2d;
-    double val3 = Double.MAX_VALUE;
-
-    assertEquals(false, bf.testDouble(val));
-    assertEquals(false, bf.testDouble(val1));
-    assertEquals(false, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(false, bf.testDouble(val1));
-    assertEquals(false, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val1);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(true, bf.testDouble(val1));
-    assertEquals(false, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val2);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(true, bf.testDouble(val1));
-    assertEquals(true, bf.testDouble(val2));
-    assertEquals(false, bf.testDouble(val3));
-    bf.addDouble(val3);
-    assertEquals(true, bf.testDouble(val));
-    assertEquals(true, bf.testDouble(val1));
-    assertEquals(true, bf.testDouble(val2));
-    assertEquals(true, bf.testDouble(val3));
-
-    double randVal = 0;
-    for (int i = 0; i < COUNT; i++) {
-      randVal = rand.nextDouble();
-      bf.addDouble(randVal);
-    }
-    // last value should be present
-    assertEquals(true, bf.testDouble(randVal));
-    // most likely this value should not exist
-    assertEquals(false, bf.testDouble(-120.2d));
-
-    assertEquals(7800, bf.sizeInBytes());
-  }
-
-  @Test
-  public void testBloomFilterString() {
-    BloomFilter bf = new BloomFilter(100000);
-    String val = "bloo";
-    String val1 = "bloom fil";
-    String val2 = "bloom filter";
-    String val3 = "cuckoo filter";
-
-    assertEquals(false, bf.testString(val));
-    assertEquals(false, bf.testString(val1));
-    assertEquals(false, bf.testString(val2));
-    assertEquals(false, bf.testString(val3));
-    bf.addString(val);
-    assertEquals(true, bf.testString(val));
-    assertEquals(false, bf.testString(val1));
-    assertEquals(false, bf.testString(val2));
-    assertEquals(false, bf.testString(val3));
-    bf.addString(val1);
-    assertEquals(true, bf.testString(val));
-    assertEquals(true, bf.testString(val1));
-    assertEquals(false, bf.testString(val2));
-    assertEquals(false, bf.testString(val3));
-    bf.addString(val2);
-    assertEquals(true, bf.testString(val));
-    assertEquals(true, bf.testString(val1));
-    assertEquals(true, bf.testString(val2));
-    assertEquals(false, bf.testString(val3));
-    bf.addString(val3);
-    assertEquals(true, bf.testString(val));
-    assertEquals(true, bf.testString(val1));
-    assertEquals(true, bf.testString(val2));
-    assertEquals(true, bf.testString(val3));
-
-    long randVal = 0;
-    for (int i = 0; i < COUNT; i++) {
-      randVal = rand.nextLong();
-      bf.addString(Long.toString(randVal));
-    }
-    // last value should be present
-    assertEquals(true, bf.testString(Long.toString(randVal)));
-    // most likely this value should not exist
-    assertEquals(false, bf.testString(Long.toString(-120)));
-
-    assertEquals(77944, bf.sizeInBytes());
-  }
-
-  @Test
-  public void testMerge() {
-    BloomFilter bf = new BloomFilter(10000);
-    String val = "bloo";
-    String val1 = "bloom fil";
-    String val2 = "bloom filter";
-    String val3 = "cuckoo filter";
-    bf.addString(val);
-    bf.addString(val1);
-    bf.addString(val2);
-    bf.addString(val3);
-
-    BloomFilter bf2 = new BloomFilter(10000);
-    String v = "2_bloo";
-    String v1 = "2_bloom fil";
-    String v2 = "2_bloom filter";
-    String v3 = "2_cuckoo filter";
-    bf2.addString(v);
-    bf2.addString(v1);
-    bf2.addString(v2);
-    bf2.addString(v3);
-
-    assertEquals(true, bf.testString(val));
-    assertEquals(true, bf.testString(val1));
-    assertEquals(true, bf.testString(val2));
-    assertEquals(true, bf.testString(val3));
-    assertEquals(false, bf.testString(v));
-    assertEquals(false, bf.testString(v1));
-    assertEquals(false, bf.testString(v2));
-    assertEquals(false, bf.testString(v3));
-
-    bf.merge(bf2);
-
-    assertEquals(true, bf.testString(val));
-    assertEquals(true, bf.testString(val1));
-    assertEquals(true, bf.testString(val2));
-    assertEquals(true, bf.testString(val3));
-    assertEquals(true, bf.testString(v));
-    assertEquals(true, bf.testString(v1));
-    assertEquals(true, bf.testString(v2));
-    assertEquals(true, bf.testString(v3));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestMurmur3.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestMurmur3.java b/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestMurmur3.java
deleted file mode 100644
index d92a3ce..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/filters/TestMurmur3.java
+++ /dev/null
@@ -1,189 +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.hadoop.hive.ql.io.filters;
-
-import static org.junit.Assert.assertEquals;
-
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.util.Random;
-
-/**
- * Tests for Murmur3 variants.
- */
-public class TestMurmur3 {
-
-  @Test
-  public void testHashCodesM3_32_string() {
-    String key = "test";
-    int seed = 123;
-    HashFunction hf = Hashing.murmur3_32(seed);
-    int hc1 = hf.hashBytes(key.getBytes()).asInt();
-    int hc2 = Murmur3.hash32(key.getBytes(), key.getBytes().length, seed);
-    assertEquals(hc1, hc2);
-
-    key = "testkey";
-    hc1 = hf.hashBytes(key.getBytes()).asInt();
-    hc2 = Murmur3.hash32(key.getBytes(), key.getBytes().length, seed);
-    assertEquals(hc1, hc2);
-  }
-
-  @Test
-  public void testHashCodesM3_32_ints() {
-    int seed = 123;
-    Random rand = new Random(seed);
-    HashFunction hf = Hashing.murmur3_32(seed);
-    for (int i = 0; i < 1000; i++) {
-      int val = rand.nextInt();
-      byte[] data = ByteBuffer.allocate(4).putInt(val).array();
-      int hc1 = hf.hashBytes(data).asInt();
-      int hc2 = Murmur3.hash32(data, data.length, seed);
-      assertEquals(hc1, hc2);
-    }
-  }
-
-  @Test
-  public void testHashCodesM3_32_longs() {
-    int seed = 123;
-    Random rand = new Random(seed);
-    HashFunction hf = Hashing.murmur3_32(seed);
-    for (int i = 0; i < 1000; i++) {
-      long val = rand.nextLong();
-      byte[] data = ByteBuffer.allocate(8).putLong(val).array();
-      int hc1 = hf.hashBytes(data).asInt();
-      int hc2 = Murmur3.hash32(data, data.length, seed);
-      assertEquals(hc1, hc2);
-    }
-  }
-
-  @Test
-  public void testHashCodesM3_32_double() {
-    int seed = 123;
-    Random rand = new Random(seed);
-    HashFunction hf = Hashing.murmur3_32(seed);
-    for (int i = 0; i < 1000; i++) {
-      double val = rand.nextDouble();
-      byte[] data = ByteBuffer.allocate(8).putDouble(val).array();
-      int hc1 = hf.hashBytes(data).asInt();
-      int hc2 = Murmur3.hash32(data, data.length, seed);
-      assertEquals(hc1, hc2);
-    }
-  }
-
-  @Test
-  public void testHashCodesM3_128_string() {
-    String key = "test";
-    int seed = 123;
-    HashFunction hf = Hashing.murmur3_128(seed);
-    // guava stores the hashcodes in little endian order
-    ByteBuffer buf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN);
-    buf.put(hf.hashBytes(key.getBytes()).asBytes());
-    buf.flip();
-    long gl1 = buf.getLong();
-    long gl2 = buf.getLong(8);
-    long[] hc = Murmur3.hash128(key.getBytes(), key.getBytes().length, seed);
-    long m1 = hc[0];
-    long m2 = hc[1];
-    assertEquals(gl1, m1);
-    assertEquals(gl2, m2);
-
-    key = "testkey128_testkey128";
-    buf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN);
-    buf.put(hf.hashBytes(key.getBytes()).asBytes());
-    buf.flip();
-    gl1 = buf.getLong();
-    gl2 = buf.getLong(8);
-    hc = Murmur3.hash128(key.getBytes(), key.getBytes().length, seed);
-    m1 = hc[0];
-    m2 = hc[1];
-    assertEquals(gl1, m1);
-    assertEquals(gl2, m2);
-  }
-
-  @Test
-  public void testHashCodesM3_128_ints() {
-    int seed = 123;
-    Random rand = new Random(seed);
-    HashFunction hf = Hashing.murmur3_128(seed);
-    for (int i = 0; i < 1000; i++) {
-      int val = rand.nextInt();
-      byte[] data = ByteBuffer.allocate(4).putInt(val).array();
-      // guava stores the hashcodes in little endian order
-      ByteBuffer buf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN);
-      buf.put(hf.hashBytes(data).asBytes());
-      buf.flip();
-      long gl1 = buf.getLong();
-      long gl2 = buf.getLong(8);
-      long[] hc = Murmur3.hash128(data, data.length, seed);
-      long m1 = hc[0];
-      long m2 = hc[1];
-      assertEquals(gl1, m1);
-      assertEquals(gl2, m2);
-    }
-  }
-
-  @Test
-  public void testHashCodesM3_128_longs() {
-    int seed = 123;
-    Random rand = new Random(seed);
-    HashFunction hf = Hashing.murmur3_128(seed);
-    for (int i = 0; i < 1000; i++) {
-      long val = rand.nextLong();
-      byte[] data = ByteBuffer.allocate(8).putLong(val).array();
-      // guava stores the hashcodes in little endian order
-      ByteBuffer buf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN);
-      buf.put(hf.hashBytes(data).asBytes());
-      buf.flip();
-      long gl1 = buf.getLong();
-      long gl2 = buf.getLong(8);
-      long[] hc = Murmur3.hash128(data, data.length, seed);
-      long m1 = hc[0];
-      long m2 = hc[1];
-      assertEquals(gl1, m1);
-      assertEquals(gl2, m2);
-    }
-  }
-
-  @Test
-  public void testHashCodesM3_128_double() {
-    int seed = 123;
-    Random rand = new Random(seed);
-    HashFunction hf = Hashing.murmur3_128(seed);
-    for (int i = 0; i < 1000; i++) {
-      double val = rand.nextDouble();
-      byte[] data = ByteBuffer.allocate(8).putDouble(val).array();
-      // guava stores the hashcodes in little endian order
-      ByteBuffer buf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN);
-      buf.put(hf.hashBytes(data).asBytes());
-      buf.flip();
-      long gl1 = buf.getLong();
-      long gl2 = buf.getLong(8);
-      long[] hc = Murmur3.hash128(data, data.length, seed);
-      long m1 = hc[0];
-      long m2 = hc[1];
-      assertEquals(gl1, m1);
-      assertEquals(gl2, m2);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
index a50def1..78d779c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.fs.PositionedReadable;
 import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.hive.common.DiskRangeList;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.io.filters.BloomFilter;
+import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.Location;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
@@ -1148,7 +1148,7 @@ public class TestRecordReaderImpl {
   public void testIntNullSafeEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong(i);
     }
@@ -1163,7 +1163,7 @@ public class TestRecordReaderImpl {
   public void testIntEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong(i);
     }
@@ -1182,7 +1182,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.INTEGER,
             "x", null, args);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong(i);
     }
@@ -1200,7 +1200,7 @@ public class TestRecordReaderImpl {
   public void testDoubleNullSafeEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addDouble(i);
     }
@@ -1215,7 +1215,7 @@ public class TestRecordReaderImpl {
   public void testDoubleEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addDouble(i);
     }
@@ -1234,7 +1234,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.FLOAT,
             "x", null, args);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addDouble(i);
     }
@@ -1252,7 +1252,7 @@ public class TestRecordReaderImpl {
   public void testStringNullSafeEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.STRING, "x", "str_15", null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addString("str_" + i);
     }
@@ -1267,7 +1267,7 @@ public class TestRecordReaderImpl {
   public void testStringEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.STRING, "x", "str_15", null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addString("str_" + i);
     }
@@ -1286,7 +1286,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.STRING,
             "x", null, args);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addString("str_" + i);
     }
@@ -1305,7 +1305,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DATE, "x", new DateWritable(15),
         null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong((new DateWritable(i)).getDays());
     }
@@ -1320,7 +1320,7 @@ public class TestRecordReaderImpl {
   public void testDateWritableEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DATE, "x", new DateWritable(15), null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong((new DateWritable(i)).getDays());
     }
@@ -1339,7 +1339,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DATE,
             "x", null, args);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong((new DateWritable(i)).getDays());
     }
@@ -1359,7 +1359,7 @@ public class TestRecordReaderImpl {
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.TIMESTAMP, "x",
         new Timestamp(15),
         null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong((new Timestamp(i)).getTime());
     }
@@ -1374,7 +1374,7 @@ public class TestRecordReaderImpl {
   public void testTimestampEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong((new Timestamp(i)).getTime());
     }
@@ -1393,7 +1393,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.TIMESTAMP,
             "x", null, args);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong((new Timestamp(i)).getTime());
     }
@@ -1413,7 +1413,7 @@ public class TestRecordReaderImpl {
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DECIMAL, "x",
         HiveDecimal.create(15),
         null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addString(HiveDecimal.create(i).toString());
     }
@@ -1429,7 +1429,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15),
         null);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addString(HiveDecimal.create(i).toString());
     }
@@ -1448,7 +1448,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
             "x", null, args);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addString(HiveDecimal.create(i).toString());
     }
@@ -1471,7 +1471,7 @@ public class TestRecordReaderImpl {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
             "x", null, args);
-    BloomFilter bf = new BloomFilter(10000);
+    BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addString(HiveDecimal.create(i).toString());
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/49f13bc7/ql/src/test/queries/clientpositive/annotate_stats_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/annotate_stats_part.q b/ql/src/test/queries/clientpositive/annotate_stats_part.q
index fcfe566..f0a68a0 100644
--- a/ql/src/test/queries/clientpositive/annotate_stats_part.q
+++ b/ql/src/test/queries/clientpositive/annotate_stats_part.q
@@ -2,6 +2,7 @@ set hive.stats.fetch.column.stats=true;
 set hive.stats.autogather=false;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.metastore.aggregate.stats.cache.enabled=false;
 
 create table if not exists loc_staging (
   state string,


Mime
View raw message