You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by to...@apache.org on 2018/12/11 13:26:44 UTC

[4/5] lucene-solr:master: Revert "LUCENE-8374 part 2/4: Reduce reads for sparse DocValues". LUCENE-8374 was committed without consensus and is expected to be superseded by LUCENE-8585.

Revert "LUCENE-8374 part 2/4: Reduce reads for sparse DocValues".
LUCENE-8374 was committed without consensus and is expected to be superseded by LUCENE-8585.

This reverts commit 7ad027627a179daa7d8d56be191d5b287dfec6f4.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/1da6d39b
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/1da6d39b
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/1da6d39b

Branch: refs/heads/master
Commit: 1da6d39b417148a3a5197931c92b6e8d409059bc
Parents: 6c5d87a
Author: Toke Eskildsen <to...@apache.org>
Authored: Tue Dec 11 14:14:22 2018 +0100
Committer: Toke Eskildsen <to...@apache.org>
Committed: Tue Dec 11 14:14:22 2018 +0100

----------------------------------------------------------------------
 .../codecs/lucene70/IndexedDISICache.java       | 102 +------
 .../lucene70/IndexedDISICacheFactory.java       |   5 -
 .../lucene/codecs/lucene70/LongCompressor.java  | 246 ---------------
 .../java/org/apache/lucene/util/RankBitSet.java | 302 -------------------
 .../codecs/lucene70/TestLongCompressor.java     |  87 ------
 .../org/apache/lucene/util/TestRankBitSet.java  | 107 -------
 6 files changed, 7 insertions(+), 842 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da6d39b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICache.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICache.java
index be0beb7..9a80689 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICache.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICache.java
@@ -33,7 +33,7 @@ import static org.apache.lucene.codecs.lucene70.IndexedDISI.MAX_ARRAY_LENGTH;
 /**
  * Caching of IndexedDISI with two strategies:
  *
- * A lookup table for block blockCache and index, and a rank structure for DENSE block lookups.
+ * A lookup table for block blockCache and index.
  *
  * The lookup table is an array of {@code long}s with an entry for each block. It allows for
  * direct jumping to the block, as opposed to iteration from the current position and forward
@@ -53,30 +53,8 @@ import static org.apache.lucene.codecs.lucene70.IndexedDISI.MAX_ARRAY_LENGTH;
  * In the case of non-existing blocks, the entry in the lookup table has index equal to the
  * previous entry and offset equal to the next non-empty block.
  *
- *
- * The rank structure for DENSE blocks is an array of unsigned {@code short}s with an entry
- * for each sub-block of 512 bits out of the 65536 bits in the outer block.
- *
- * Each rank-entry states the number of set bits within the block up to the bit before the
- * bit positioned at the start of the sub-block.
- * Note that that the rank entry of the first sub-block is always 0 and that the last entry can
- * at most be 65536-512 = 65024 and thus will always fit into an unsigned short.
- *
- * See https://en.wikipedia.org/wiki/Succinct_data_structure for details on rank structures.
- * The alternative to using the rank structure is iteration and summing of set bits for all
- * entries in the DENSE sub-block up until the wanted bit, with a worst-case of 1024 entries.
- * The rank cache overhead for a single DENSE block is 128 shorts (128*16 = 2048 bits) or
- * 1/32th.
- *
- * The ranks for the DENSE blocks are stored in a structure shared for the whole array of
- * blocks, DENSE or not. To avoid overhead that structure is itself sparse. See
- * {@link LongCompressor} for details on DENSE structure sparseness.
- *
- *
- * The performance overhead for creating a cache instance is equivalent to accessing all
- * DocValues values for the given field, i.e. it scales lineary to field size. On modern
- * hardware it is in the ballpark of 1ms for 5M values on modern hardware. Caveat lector:
- * At the point of writing, performance points are only available for 2 real-world setups.
+ * The performance overhead for creating a cache instance is equivalent to visiting every 65536th
+ * doc value for the given field, i.e. it scales lineary to field size.
  */
 public class IndexedDISICache implements Accountable {
   private static final int BLOCK = 65536;   // The number of docIDs that a single block represents
@@ -85,12 +63,6 @@ public class IndexedDISICache implements Accountable {
   private static final long BLOCK_INDEX_MASK = ~0L << BLOCK_INDEX_SHIFT; // The index bits in a lookup entry
   private static final long BLOCK_LOOKUP_MASK = ~BLOCK_INDEX_MASK; // The offset bits in a lookup entry
 
-  private static final int RANK_BLOCK = 512; // The number of docIDs/bits in each rank-sub-block within a DENSE block
-  static final int RANK_BLOCK_LONGS = 512/Long.SIZE; // The number of longs making up a rank-block (8)
-  private static final int RANK_BLOCK_BITS = 9;
-  private static final int RANKS_PER_BLOCK = BLOCK/RANK_BLOCK;
-
-  private PackedInts.Reader rank;   // One every 512 docs, sparsely represented as not all blocks are DENSE
   private long[] blockCache = null; // One every 65536 docs, contains index & slice position
   private String creationStats = "";
   private final String name; // Identifier for debug, log & inspection
@@ -114,7 +86,6 @@ public class IndexedDISICache implements Accountable {
 
   private IndexedDISICache() {
     this.blockCache = null;
-    this.rank = null;
     this.name = "";
   }
 
@@ -146,45 +117,10 @@ public class IndexedDISICache implements Accountable {
         -1 : (int)(blockCache[targetBlock] >>> BLOCK_INDEX_SHIFT);
   }
 
-  /**
-   * Given a target (docID), this method returns the first docID in the entry containing the target.
-   * @param target the docID for which an index is wanted.
-   * @return the docID where the rank is known. This will be lte target.
-   */
-  // TODO: This method requires a lot of knowledge of the intrinsics of the cache. Usage should be simplified
-  int denseRankPosition(int target) {
-       return target >> RANK_BLOCK_BITS << RANK_BLOCK_BITS;
-  }
-
   public boolean hasOffsets() {
     return blockCache != null;
   }
 
-  boolean hasRank() {
-    return rank != null;
-  }
-  
-  /**
-   * Get the rank (index) for all set bits up to just before the given rankPosition in the block.
-   * The caller is responsible for deriving the count of bits up to the docID target from the rankPosition.
-   * The caller is also responsible for keeping track of set bits up to the current block.
-   * Important: This only accepts rankPositions that aligns to {@link #RANK_BLOCK} boundaries.
-   * Note 1: Use {@link #denseRankPosition(int)} to obtain a calid rankPosition for a wanted docID.
-   * Note 2: The caller should seek to the rankPosition in the underlying slice to keep everything in sync.
-   * @param rankPosition a docID target that aligns to {@link #RANK_BLOCK}.
-   * @return the rank (index / set bits count) up to just before the given rankPosition.
-   *         If rank is disabled, -1 is returned.
-   */
-  // TODO: This method requires a lot of knowledge of the intrinsics of the cache. Usage should be simplified
-  int getRankInBlock(int rankPosition) {
-    if (rank == null) {
-      return -1;
-    }
-    assert rankPosition == denseRankPosition(rankPosition);
-    int rankIndex = rankPosition >> RANK_BLOCK_BITS;
-    return rankIndex >= rank.size() ? -1 : (int) rank.get(rankIndex);
-  }
-
   private void updateCaches(IndexInput slice) throws IOException {
     final long startOffset = slice.getFilePointer();
 
@@ -199,19 +135,17 @@ public class IndexedDISICache implements Accountable {
 
     slice.seek(startOffset); // Leave it as we found it
     creationStats = String.format(Locale.ENGLISH,
-        "name=%s, blocks=%d (ALL=%d, DENSE=%d, SPARSE=%d, EMPTY=%d), time=%dms, block=%d bytes, rank=%d bytes",
+        "name=%s, blocks=%d (ALL=%d, DENSE=%d, SPARSE=%d, EMPTY=%d), time=%dms, block=%d bytes",
         name,
         largestBlock+1, statBlockALL.get(), statBlockDENSE.get(), statBlockSPARSE.get(),
         (largestBlock+1-statBlockALL.get()-statBlockDENSE.get()-statBlockSPARSE.get()),
         (System.nanoTime()-startTime)/1000000,
-        blockCache == null ? 0 : blockCache.length*Long.BYTES,
-        rank == null ? 0 : rank.ramBytesUsed());
+        blockCache == null ? 0 : blockCache.length*Long.BYTES);
   }
 
   private int fillCache(
       IndexInput slice, AtomicInteger statBlockALL, AtomicInteger statBlockDENSE, AtomicInteger statBlockSPARSE)
       throws IOException {
-    char[] buildRank = new char[256];
     int largestBlock = -1;
     long index = 0;
     int rankIndex = -1;
@@ -246,27 +180,7 @@ public class IndexedDISICache implements Accountable {
       // The block is DENSE
       statBlockDENSE.incrementAndGet();
       long nextBlockOffset = slice.getFilePointer() + (1 << 13);
-      int setBits = 0;
-      int rankOrigo = blockIndex << 16 >> 9; // Double shift for clarity: The compiler will simplify it
-      for (int rankDelta = 0 ; rankDelta < RANKS_PER_BLOCK ; rankDelta++) { // 128 rank-entries in a block
-        rankIndex = rankOrigo + rankDelta;
-        buildRank = ArrayUtil.grow(buildRank, rankIndex+1);
-        buildRank[rankIndex] = (char)setBits;
-        for (int i = 0 ; i < 512/64 ; i++) { // 8 longs for each rank-entry
-          setBits += Long.bitCount(slice.readLong());
-        }
-      }
-      assert slice.getFilePointer() == nextBlockOffset;
-    }
-    // Compress the buildRank as it is potentially very sparse
-    if (rankIndex < 0) {
-      rank = null;
-    } else {
-      PackedInts.Mutable ranks = PackedInts.getMutable(rankIndex, 16, PackedInts.DEFAULT); // Char = 16 bit
-      for (int i = 0 ; i < rankIndex ; i++) {
-        ranks.set(i, buildRank[i]);
-      }
-      rank = LongCompressor.compress(ranks);
+      slice.seek(nextBlockOffset);
     }
 
     return largestBlock;
@@ -275,7 +189,6 @@ public class IndexedDISICache implements Accountable {
   private void freezeCaches(int largestBlock) {
     if (largestBlock == -1) { // No set bit: Disable the caches
       blockCache = null;
-      rank = null;
       return;
     }
 
@@ -315,8 +228,7 @@ public class IndexedDISICache implements Accountable {
   @Override
   public long ramBytesUsed() {
     return (blockCache == null ? 0 : RamUsageEstimator.sizeOf(blockCache)) +
-        (rank == null ? 0 : rank.ramBytesUsed()) +
         RamUsageEstimator.NUM_BYTES_OBJECT_REF*3 +
         RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + creationStats.length()*2;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da6d39b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICacheFactory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICacheFactory.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICacheFactory.java
index 7a85727..6cf8af1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICacheFactory.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICacheFactory.java
@@ -118,8 +118,6 @@ public class IndexedDISICacheFactory implements Accountable {
     IndexedDISICache cache = disiPool.get(key);
     if (cache == null) {
       // TODO: Avoid overlapping builds of the same cache
-      // Both BLOCK & DENSE caches are created as they might be requested later for the field,
-      // regardless of whether they are requested now
       cache = new IndexedDISICache(slice, name);
       disiPool.put(key, cache);
     }
@@ -130,9 +128,6 @@ public class IndexedDISICacheFactory implements Accountable {
   public long getDISIBlocksWithOffsetsCount() {
     return disiPool.values().stream().filter(IndexedDISICache::hasOffsets).count();
   }
-  public long getDISIBlocksWithRankCount() {
-    return disiPool.values().stream().filter(IndexedDISICache::hasRank).count();
-  }
 
   @Override
   public long ramBytesUsed() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da6d39b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/LongCompressor.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/LongCompressor.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/LongCompressor.java
deleted file mode 100644
index 1c030a7..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/LongCompressor.java
+++ /dev/null
@@ -1,246 +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.lucene.codecs.lucene70;
-
-import org.apache.lucene.util.MathUtil;
-import org.apache.lucene.util.RankBitSet;
-import org.apache.lucene.util.packed.PackedInts;
-
-/**
- * Utility class for generating compressed read-only in-memory representations of longs.
- * The representation is optimized towards random access primarily and space secondarily.
- *
- * The representation always applies delta-to-minvalue and greatest-common-divisor compression.
- *
- * Depending on the number of 0-entries and the length of the array, a sparse representation is
- * used, using rank to improve access speed. Sparseness introduces an O(1) access time overhead.
- * Sparseness can be turned off.
- */
-public class LongCompressor {
-
-  /**
-   * The minimum amount of total values in the data set for sparse to be active.
-   */
-  private static final int DEFAULT_MIN_TOTAL_VALUES_FOR_SPARSE = 10_000;
-
-  /**
-   * The minimum total amount of zero values in the data set for sparse to be active.
-   */
-  private static final int DEFAULT_MIN_ZERO_VALUES_FOR_SPARSE = 500;
-
-  /**
-   * The minimum fraction of the data set that must be zero for sparse to be active.
-   */
-  private static final double DEFAULT_MIN_ZERO_VALUES_FRACTION_FOR_SPARSE = 0.2; // 20% (just guessing of a value here)
-
-  /**
-   * Create a compact version of the given values.
-   * @param values PackedInts with no special constraints.
-   * @return a compact version of the given values or the given values if compression did not improve on heap overhead.
-   */
-  public static PackedInts.Reader compress(PackedInts.Reader values) {
-    return compress(values, values.size());
-  }
-
-  /**
-   * Create a compact version of the given values from index 0 to length-1.
-   * @param values PackedInts with no special constraints.
-   * @param length the number of values to compress.
-   * @return a compact version of the given values or the given values if compression did not improve on heap overhead.
-   */
-  public static PackedInts.Reader compress(PackedInts.Reader values, int length) {
-    return compress(values, values.size(), true);
-  }
-
-  /**
-   * Create a compact version of the given values from index 0 to length-1.
-   * @param values PackedInts with no special constraints.
-   * @param length the number of values to compress.
-   * @param allowSparse if true and is the default limits matches the input, a sparse representation will be created.
-   * @return a compact version of the given values or the given values if compression did not improve on heap overhead.
-   */
-  public static PackedInts.Reader compress(PackedInts.Reader values, int length, boolean allowSparse) {
-    return compress(values, length, allowSparse,
-        DEFAULT_MIN_TOTAL_VALUES_FOR_SPARSE,
-        DEFAULT_MIN_ZERO_VALUES_FOR_SPARSE,
-        DEFAULT_MIN_ZERO_VALUES_FRACTION_FOR_SPARSE);
-  }
-
-  /**
-   * Create a compact version of the given values from index 0 to length-1.
-   * @param values PackedInts with no special constraints.
-   * @param length the number of values to compress.
-   * @param allowSparse if true and is the default limits matches the input, a sparse representation will be created.
-   * @param minTotalSparse the minimum absolute number of 0-entries needed for a sparse representation.
-   *                       0-entries are counted after minValue compression: {@code 3, 5, 3, 7, 16} has two 0-entries.
-   * @return a compact version of the given values or the given values if compression did not improve on heap overhead.
-   */
-  public static PackedInts.Reader compress(
-      PackedInts.Reader values, int length, boolean allowSparse,
-      int minTotalSparse, int minZeroSparse, double minZeroFractionSparse) {
-    if (length == 0) {
-      return PackedInts.getMutable(0, 1, PackedInts.DEFAULT);
-    }
-
-    final long min = getMin(values, length);
-    final long gcd = getGCD(values, length, min);
-    final long maxCompressed = getMax(values, length, min, gcd);
-
-    int zeroCount;
-    if (!isPossiblySparseCandidate(length, allowSparse, minTotalSparse) ||
-        !isSparseCandidate(values, length, true, minTotalSparse,
-            (zeroCount = countZeroes(values, length, min, gcd)), minZeroSparse, minZeroFractionSparse)) {
-      // TODO: Add abort-early if it becomes apparent that no space saving is possible
-      PackedInts.Mutable inner =
-          PackedInts.getMutable(length, PackedInts.bitsRequired(maxCompressed), PackedInts.DEFAULT);
-      for (int i = 0 ; i < length ; i++) {
-        inner.set(i, (values.get(i)-min)/gcd);
-      }
-      PackedInts.Reader comp = new CompressedReader(inner, min, gcd);
-      // Sanity check that compression worked and if not, return the original input
-      return comp.ramBytesUsed() < values.ramBytesUsed() ? comp : values;
-    }
-
-    // Sparsify
-    RankBitSet rank = new RankBitSet(length);
-    PackedInts.Mutable inner =
-        PackedInts.getMutable(values.size()-zeroCount, PackedInts.bitsRequired(maxCompressed), PackedInts.DEFAULT);
-    int valueIndex = 0;
-    for (int i = 0 ; i < length ; i++) {
-      long value = (values.get(i)-min)/gcd;
-      if (value != 0) {
-        rank.set(i);
-        inner.set(valueIndex++, value);
-      }
-    }
-    rank.buildRankCache();
-    PackedInts.Reader comp = new CompressedReader(inner, min, gcd, rank);
-    // Sanity check that compression worked and if not, return the original input
-    return comp.ramBytesUsed() < values.ramBytesUsed() ? comp : values;
-  }
-
-  // Fast check
-  private static boolean isPossiblySparseCandidate(int length, boolean allowSparse, int minTotalSparse) {
-    return allowSparse && minTotalSparse <= length;
-  }
-
-  // Also fast, but requires zeroCount which is slow to calculate
-  private static boolean isSparseCandidate(
-      PackedInts.Reader values, int length, boolean allowSparse, int minTotalSparse,
-      int zeroCount, int minZeroSparse, double minZeroFractionSparse) {
-    return allowSparse && minTotalSparse <= length &&
-        minZeroSparse < zeroCount && minZeroFractionSparse < 1.0 * zeroCount / length;
-  }
-
-  // Not very fast as is requires #length divisions.
-  private static int countZeroes(PackedInts.Reader values, int length, long min, final long gcd) {
-    int zeroCount = 0;
-    for (int i = 0 ; i < length ; i++) {
-      if ((values.get(i)-min)/gcd == 0) { // Hope the case where gcd==1 gets JITted. We could add a switch to be sure?
-        zeroCount++;
-      }
-    }
-    return zeroCount;
-  }
-
-  static class CompressedReader extends PackedInts.Reader {
-    private final PackedInts.Reader inner;
-    final long min;
-    final long gcd;
-    final RankBitSet rank;
-
-    CompressedReader(PackedInts.Reader inner, long min, long gcd) {
-      this(inner, min, gcd, null);
-    }
-
-    CompressedReader(PackedInts.Reader inner, long min, long gcd, RankBitSet rank) {
-      this.inner = inner;
-      this.min = min;
-      this.gcd = gcd;
-      this.rank = rank;
-    }
-
-    @Override
-    public int size() {
-      return rank == null ? inner.size() : rank.length();
-    }
-
-    @Override
-    public long get(int docID) {
-      // No rank: The value at the index
-      // Rank but no set bit: min*gcd
-      // Rank and set bit: (The value at the rank + min) * gcd
-      return (rank == null ? inner.get(docID) : rank.get(docID) ? inner.get(rank.rank(docID)) : 0) * gcd + min;
-    }
-
-    @Override
-    public long ramBytesUsed() {
-      return inner.ramBytesUsed() + (rank == null ? 0 : rank.ramBytesUsed());
-    }
-  }
-
-  private static long getMin(PackedInts.Reader values, int length) {
-    long min = Long.MAX_VALUE;
-    for (int i = 0 ; i < length ; i++) {
-      if (min > values.get(i)) {
-        min = values.get(i);
-      }
-    }
-    return min;
-  }
-
-  // GCD-code takes & adjusted from Lucene70DocValuesConsumer
-  private static long getGCD(final PackedInts.Reader values, final int length, final long min) {
-    long gcd = -1;
-
-    for (int i = 0 ; i < length ; i++) {
-      long value = values.get(i)-min;
-      if (value == 0) {
-        continue;
-      }
-      if (gcd == -1) {
-        gcd = value;
-        continue;
-      }
-
-      if (value < Long.MIN_VALUE / 2 || value > Long.MAX_VALUE / 2) {
-        // in that case v - minValue might overflow and make the GCD computation return
-        // wrong results. Since these extreme values are unlikely, we just discard
-        // GCD computation for them
-        gcd = 1;
-      } else { // minValue needs to be set first
-        gcd = MathUtil.gcd(gcd, value);
-      }
-
-      if (gcd == 1) {
-        break;
-      }
-    }
-    return gcd == -1 ? 1 : gcd;
-  }
-
-  private static long getMax(final PackedInts.Reader values, final int length, final long min, final long gcd) {
-    long rawMax = Long.MIN_VALUE;
-    for (int i = 0 ; i < length ; i++) {
-      long value = values.get(i);
-      if (value > rawMax) {
-        rawMax = value;
-      }
-    }
-    return (rawMax-min)/gcd;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da6d39b/lucene/core/src/java/org/apache/lucene/util/RankBitSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/RankBitSet.java b/lucene/core/src/java/org/apache/lucene/util/RankBitSet.java
deleted file mode 100644
index 3702c63..0000000
--- a/lucene/core/src/java/org/apache/lucene/util/RankBitSet.java
+++ /dev/null
@@ -1,302 +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.lucene.util;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.lucene.search.DocIdSetIterator;
-
-/**
- * Wrapper for OpenBitSet which creates and exposes a rank cache. The rank-cache scales to 2 billion bits.
- *
- * The rankCache has a long for every 2048 bits and thus has an overhead of 3.17% relative to the given bit set.
- * Performance is O(1):
- * 1 lookup in cache,
- * a maximum of 3 sums,
- * a maximum of 8 Long.bitCounts.
- *
- * Creation performance is equivalent to a full count of all set bits in the bit set O(n).
- *
- * Note: {@link #buildRankCache()} must be called once after the bit set has been created or changed and before
- * calling {@link #rank(long)}.
- *
- * The design is based heavily on the article
- * Space-Efficient, High-Performance Rank and Select Structures on Uncompressed Bit Sequences
- * by Dong Zhou, David G. Andersen, Michael Kaminsky, Carnegie Mellon University, Intel Labs
- * http://www.cs.cmu.edu/~dga/papers/zhou-sea2013.pdf
- */
-// Note: If the total number of set bits is <= 65535, a faster rank cache would be a short for every 512 bits
-// This is not available in the current implementation.
-// Extending the rank beyond 2 billion bits could be done by dividing the bitmap into blocks of 2b bits and
-// introducing yet another table with a rank-origo for each 2b-block
-public class RankBitSet extends BitSet {
-  public static final int  LOWER_BITS = 32; // Must be capable of addressing full Java array
-  public static final long LOWER_MASK = ~(~1L << (LOWER_BITS-1));
-  public static final int LOWER_OVER_BITS = 11;
-  public static final long LOWER_OVER_MASK = ~(~1L << (LOWER_OVER_BITS-1));
-  public static final int LOWER_OVER_SIZE = 2048; // Overflow bits handled by a lower block
-
-  public static final int BASIC_BITS = 10; // Needs to hold counts from 0-512 (513-1023 are never used)
-  public static final long BASIC_MASK = ~(~1L << (BASIC_BITS-1));
-  public static final int BASIC_OVER_BITS = 9;
-  public static final long BASIC_OVER_MASK = ~(~1L << (BASIC_OVER_BITS-1));
-  public static final int BASIC_OVER_SIZE = 512; // Overflow bits handled by a basic block
-  public static final int BASIC_WORDS = BASIC_OVER_SIZE /Long.SIZE; // word == long
-  /**
-   * Each entry is made up of 1 long<br/>
-   * Bits 63-32: 32 bit first-level absolute index.<br/>
-   * Bits 30+31 are unused. These could be used to signal all-set or all-unset for the block to spare a few cycles?
-   * Bits 29-0: 3 * 10 bit (0-1023) second-level relative index. Only numbers 0-512 are used.
-   */
-  private long[] rankCache = null;
-  private final FixedBitSet inner;
-  private final int wlen; // Number of words (longs with FixedBitSet) in inner
-
-  public RankBitSet(int numBits) {
-    this(new FixedBitSet(numBits));
-  }
-
-  public RankBitSet(FixedBitSet inner) {
-    this.inner = inner;
-    wlen = inner.getBits().length;
-  }
-
-  /**
-   * Must be called after bits has changed and before {@link #rank} is called.
-   */
-  public void buildRankCache() {
-    rankCache = new long[(length() >>> LOWER_OVER_BITS)+1];
-    long total = 0;
-    int lower = 0 ;
-    while (lower * LOWER_OVER_SIZE < length()) { // Full lower block processing
-      final int origoWordIndex = (lower * LOWER_OVER_SIZE) >>> 6;
-    // TODO: Some conditionals could be spared by checking once if all basic blocks are within size
-      final long basic1 = origoWordIndex + BASIC_WORDS <= wlen ?
-          BitUtil.pop_array(inner.getBits(), origoWordIndex, BASIC_WORDS) : 0;
-      final long basic2 =  origoWordIndex + BASIC_WORDS*2 <= wlen ?
-          BitUtil.pop_array(inner.getBits(), origoWordIndex + BASIC_WORDS, BASIC_WORDS) : 0;
-      final long basic3 =  origoWordIndex + BASIC_WORDS*3 <= wlen ?
-          BitUtil.pop_array(inner.getBits(), origoWordIndex + BASIC_WORDS *2, BASIC_WORDS) : 0;
-      final long basic4 =  origoWordIndex + BASIC_WORDS*4 <= wlen ?
-          BitUtil.pop_array(inner.getBits(), origoWordIndex + BASIC_WORDS *3, BASIC_WORDS) : 0;
-      rankCache[lower] = total << (Long.SIZE-LOWER_BITS) |
-           basic1 << (BASIC_BITS *2) |
-           basic2 << BASIC_BITS |
-           basic3;
-      total += basic1 + basic2 + basic3 + basic4;
-      lower++;
-    }
-  }
-
-  /**
-   * Get the rank (number of set bits up to right before the index) for the given index in O(1).
-   * @param index offset in the originating bit set.
-   * @return the rank for the index.
-   */
-  public int rank(long index) {
-    final long cache = rankCache[((int) (index >>> LOWER_OVER_BITS))];
-    // lower cache (absolute)
-    long rank = cache >>> (Long.SIZE-LOWER_BITS);
-    int startBitIndex = (int) (index & ~LOWER_OVER_MASK);
-    // basic blocks (relative)
-    if (startBitIndex < index-BASIC_OVER_SIZE) {
-      rank += (cache >>> (BASIC_BITS*2)) & BASIC_MASK;
-      startBitIndex += BASIC_OVER_SIZE;
-      if (startBitIndex < index-BASIC_OVER_SIZE) {
-        rank += (cache >>> BASIC_BITS) & BASIC_MASK;
-        startBitIndex += BASIC_OVER_SIZE;
-        if (startBitIndex < index-BASIC_OVER_SIZE) {
-          rank += cache & BASIC_MASK;
-          startBitIndex += BASIC_OVER_SIZE;
-        }
-      }
-    }
-    // long.bitcount (relative)
-    while(startBitIndex < index-Long.SIZE) {
-      rank += Long.bitCount(inner.getBits()[startBitIndex >>> 6]);
-      startBitIndex += Long.SIZE;
-    }
-    // Single bits (relative)
-    if (startBitIndex < index) {
-/*      System.out.println(String.format(Locale.ENGLISH,
-          "startBitIndex=%d, index=%d, getBits()[startBitIndex>>>6=%d]=%s, index-startBitIndex=%d, mask=%s",
-          startBitIndex, index, startBitIndex>>>6, Long.toBinaryString(getBits()[startBitIndex>>>6]),
-          index-startBitIndex, Long.toBinaryString(~(~1L << (index-startBitIndex-1)))));*/
-      rank += Long.bitCount(inner.getBits()[startBitIndex >>> 6] & ~(~1L << (index-startBitIndex-1)));
-    }
-//    for (int i = startBitIndex ; i < index ; i++) {
-//      rank += fastGet(i) ? 1 : 0;
-//    }
-    return (int) rank;
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_OBJECT_REF*2 +
-        Integer.BYTES + Long.BYTES) +
-        inner.ramBytesUsed() +
-        (rankCache == null ? 0 :
-            RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + Long.BYTES*rankCache.length);
-  }
-
-  /* Delegations to inner bit set below */
-
-  public static FixedBitSet ensureCapacity(FixedBitSet bits, int numBits) {
-    return FixedBitSet.ensureCapacity(bits, numBits);
-  }
-
-  public static int bits2words(int numBits) {
-    return FixedBitSet.bits2words(numBits);
-  }
-
-  public static long intersectionCount(FixedBitSet a, FixedBitSet b) {
-    return FixedBitSet.intersectionCount(a, b);
-  }
-
-  public static long unionCount(FixedBitSet a, FixedBitSet b) {
-    return FixedBitSet.unionCount(a, b);
-  }
-
-  public static long andNotCount(FixedBitSet a, FixedBitSet b) {
-    return FixedBitSet.andNotCount(a, b);
-  }
-
-  @Override
-  public int length() {
-    return inner.length();
-  }
-
-  public long[] getBits() {
-    return inner.getBits();
-  }
-
-  @Override
-  public int cardinality() {
-    return inner.cardinality();
-  }
-
-  @Override
-  public boolean get(int index) {
-    return inner.get(index);
-  }
-
-  @Override
-  public void set(int index) {
-    inner.set(index);
-  }
-
-  public boolean getAndSet(int index) {
-    return inner.getAndSet(index);
-  }
-
-  @Override
-  public void clear(int index) {
-    inner.clear(index);
-  }
-
-  public boolean getAndClear(int index) {
-    return inner.getAndClear(index);
-  }
-
-  @Override
-  public int nextSetBit(int index) {
-    return inner.nextSetBit(index);
-  }
-
-  @Override
-  public int prevSetBit(int index) {
-    return inner.prevSetBit(index);
-  }
-
-  @Override
-  public void or(DocIdSetIterator iter) throws IOException {
-    inner.or(iter);
-  }
-
-  public void or(FixedBitSet other) {
-    inner.or(other);
-  }
-
-  public void xor(FixedBitSet other) {
-    inner.xor(other);
-  }
-
-  public void xor(DocIdSetIterator iter) throws IOException {
-    inner.xor(iter);
-  }
-
-  public boolean intersects(FixedBitSet other) {
-    return inner.intersects(other);
-  }
-
-  public void and(FixedBitSet other) {
-    inner.and(other);
-  }
-
-  public void andNot(FixedBitSet other) {
-    inner.andNot(other);
-  }
-
-  public boolean scanIsEmpty() {
-    return inner.scanIsEmpty();
-  }
-
-  public void flip(int startIndex, int endIndex) {
-    inner.flip(startIndex, endIndex);
-  }
-
-  public void flip(int index) {
-    inner.flip(index);
-  }
-
-  public void set(int startIndex, int endIndex) {
-    inner.set(startIndex, endIndex);
-  }
-
-  @Override
-  public void clear(int startIndex, int endIndex) {
-    inner.clear(startIndex, endIndex);
-  }
-
-  @Override
-  public int hashCode() {
-    return inner.hashCode();
-  }
-
-  public static FixedBitSet copyOf(Bits bits) {
-    return FixedBitSet.copyOf(bits);
-  }
-
-  public Bits asReadOnlyBits() {
-    return inner.asReadOnlyBits();
-  }
-
-  public static BitSet of(DocIdSetIterator it, int maxDoc) throws IOException {
-    return BitSet.of(it, maxDoc);
-  }
-
-  @Override
-  public int approximateCardinality() {
-    return inner.approximateCardinality();
-  }
-
-  @Override
-  public Collection<Accountable> getChildResources() {
-    return inner.getChildResources();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da6d39b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLongCompressor.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLongCompressor.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLongCompressor.java
deleted file mode 100644
index bb87dc4..0000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLongCompressor.java
+++ /dev/null
@@ -1,87 +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.lucene.codecs.lucene70;
-
-import java.util.Locale;
-import java.util.Random;
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.packed.PackedInts;
-
-public class TestLongCompressor extends LuceneTestCase {
-
-  // Simple money test that builds a collection of random longs, compresses them with LongCompressor and
-  // checks if all values from the compressed version are equal to the source
-  public void testLongCompressorMonkey() {
-    final int RUNS = 10;
-    final int[] MAX_SIZES = new int[]{0, 1, 10, 1000, 100_000};
-    final int[] MAX_VALUE = new int[]{0, 1, 10, 1000, 100_000};
-    for (int run = 0 ; run < RUNS ; run++) {
-      for (int maxSize: MAX_SIZES) {
-        int size = maxSize == 0 ? 0 : random().nextInt(maxSize);
-        for (int maxValue: MAX_VALUE) {
-          int minValue = maxValue == 0 ? 0 : random().nextInt(maxValue);
-          double minChance = random().nextDouble();
-          longCompressorMonkeyTest(run, size, minValue, maxValue, minChance, random().nextLong());
-        }
-      }
-    }
-  }
-
-  public void testVerySparse() {
-    final int SIZE = 674932;
-    final int EVERY = SIZE/896;
-    PackedInts.Mutable ranks = PackedInts.getMutable(674932, 16, PackedInts.DEFAULT);
-    for (int i = 0 ; i < SIZE; i+=EVERY) {
-      ranks.set(i, random().nextInt(65535));
-    }
-    PackedInts.Reader sparsed = LongCompressor.compress(ranks);
-    assertFalse("The input and the sparsed should not be the same", ranks == sparsed);
-  }
-
-  private void longCompressorMonkeyTest(
-      int run, int size, int minValue, int maxValue, double minChance, long randomSeed) {
-    final String description = String.format(Locale.ENGLISH,
-        "run=%d, size=%d, minValue=%d, maxValue=%d, minChance=%1.2f, seed=%d",
-        run, size, minValue, maxValue, minChance, randomSeed);
-    Random innerRandom = new Random(randomSeed);
-    PackedInts.Mutable expected = PackedInts.getMutable(size, PackedInts.bitsRequired(maxValue), PackedInts.DEFAULT);
-    for (int i = 0 ; i < size ; i++) {
-      if (innerRandom.nextDouble() <= minChance) {
-        continue;
-      }
-      expected.set(i, maxValue-minValue == 0 ? minValue : innerRandom.nextInt(maxValue-minValue)+minValue);
-    }
-    assertSparseValid(description, expected);
-  }
-
-  private void assertSparseValid(String description, PackedInts.Reader values) {
-    try {
-      PackedInts.Reader sparsed = LongCompressor.compress(values, values.size());
-
-      for (int i = 0; i < values.size(); i++) {
-        assertEquals("The value at index " + i + " should be as expected for " + description,
-            values.get(i), sparsed.get(i));
-      }
-    } catch (Exception e) {
-      throw new RuntimeException("Unexpected Exception for " + description, e);
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1da6d39b/lucene/core/src/test/org/apache/lucene/util/TestRankBitSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestRankBitSet.java b/lucene/core/src/test/org/apache/lucene/util/TestRankBitSet.java
deleted file mode 100644
index 677cc22..0000000
--- a/lucene/core/src/test/org/apache/lucene/util/TestRankBitSet.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.lucene.util;
-
-import java.util.Locale;
-import java.util.Random;
-
-public class TestRankBitSet extends LuceneTestCase {
-
-  public void testSingleWord() {
-    RankBitSet rank = new RankBitSet(60);
-    rank.set(20);
-    rank.buildRankCache();
-
-    assertEquals("The rank at index 20 should be correct", 0, rank.rank(20));
-    assertEquals("The rank at index 21 should be correct", 1, rank.rank(21));
-  }
-
-  public void testSecondWord() {
-    RankBitSet rank = new RankBitSet(100);
-    rank.set(70);
-    rank.buildRankCache();
-
-    assertEquals("The rank at index 70 should be correct", 0, rank.rank(70));
-    assertEquals("The rank at index 71 should be correct", 1, rank.rank(71));
-  }
-
-  public void testThirdWord() {
-    RankBitSet rank = new RankBitSet(200);
-    rank.set(130);
-    rank.buildRankCache();
-
-    assertEquals("The rank at index 130 should be correct", 0, rank.rank(130));
-    assertEquals("The rank at index 131 should be correct", 1, rank.rank(131));
-  }
-
-  public void testSecondLower() {
-    RankBitSet rank = new RankBitSet(3000);
-    rank.set(2500);
-    rank.buildRankCache();
-
-    assertEquals("The rank at index 2500 should be correct", 0, rank.rank(2500));
-    assertEquals("The rank at index 2500 should be correct", 1, rank.rank(2501));
-  }
-
-  public void testSpecific282() {
-    RankBitSet rank = new RankBitSet(448);
-    rank.set(282);
-    rank.buildRankCache();
-
-    assertEquals("The rank at index 288 should be correct", 1, rank.rank(288));
-  }
-
-  public void testSpecific1031() {
-    RankBitSet rank = new RankBitSet(1446);
-    rank.set(1031);
-    rank.buildRankCache();
-
-    assertEquals("The rank at index 1057 should be correct", 1, rank.rank(1057));
-  }
-
-  public void testMonkeys() {
-    monkey(20, 8000, 40);
-  }
-
-  @Slow
-  public void testManyMonkeys() {
-    monkey(20, 100000, 400);
-  }
-
-  public void monkey(int runs, int sizeMax, int setMax) {
-    Random random = random();
-    //Random random = new Random(87);
-    for (int run = 0 ; run < runs ; run++) {
-      final int size = random.nextInt(sizeMax-1)+1;
-      RankBitSet rank = new RankBitSet(size);
-      int doSet = random.nextInt(setMax);
-      for (int s = 0 ; s < doSet ; s++) {
-        int index = random.nextInt(size);
-        rank.set(index);
-      }
-      rank.buildRankCache();
-      int setbits = 0;
-      for (int i = 0 ; i < size ; i++) {
-        assertEquals(String.format(Locale.ENGLISH, "run=%d, index=%d/%d, setbits=%d", run, i, size, setbits),
-            setbits, rank.rank(i));
-        if (rank.get(i)) {
-          setbits++;
-        }
-      }
-    }
-  }
-}