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:45 UTC

[5/5] lucene-solr:master: Revert "LUCENE-8374 part 1/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 1/4: Reduce reads for sparse DocValues".
LUCENE-8374 was committed without consensus and is expected to be superseded by LUCENE-8585.

This reverts commit 58a7a8ada5cebeb261060c56cd6d0a9446478bf6.


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

Branch: refs/heads/master
Commit: 8a20705b82272352ffcef8a18a7e8f96b2c05a7b
Parents: 1da6d39
Author: Toke Eskildsen <to...@apache.org>
Authored: Tue Dec 11 14:17:57 2018 +0100
Committer: Toke Eskildsen <to...@apache.org>
Committed: Tue Dec 11 14:17:57 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 -
 .../lucene/codecs/lucene70/IndexedDISI.java     |  65 +-----
 .../codecs/lucene70/IndexedDISICache.java       | 234 -------------------
 .../lucene70/IndexedDISICacheFactory.java       | 150 ------------
 .../lucene70/Lucene70DocValuesProducer.java     |  85 ++-----
 .../lucene/codecs/lucene70/TestIndexedDISI.java |  91 ++------
 .../org/apache/lucene/index/TestDocValues.java  | 105 ---------
 7 files changed, 49 insertions(+), 684 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a20705b/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 22e3e61..212607f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -220,9 +220,6 @@ Optimizations
   to early terminate the iterator if the minimum score is greater than the constant
   score. (Christophe Bismuth via Jim Ferenczi)
 
-* LUCENE-8374: Reduce reads for sparse DocValues and whole number numeric DocValues.
-  (Toke Eskildsen)
-
 ======================= Lucene 7.7.0 =======================
 
 Build

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a20705b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java
index 114710e..6138896 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISI.java
@@ -50,9 +50,6 @@ import org.apache.lucene.util.RoaringDocIdSet;
 final class IndexedDISI extends DocIdSetIterator {
 
   static final int MAX_ARRAY_LENGTH = (1 << 12) - 1;
-  static final String NO_NAME = "n/a";
-
-  public final String name;
 
   private static void flush(int block, FixedBitSet buffer, int cardinality, IndexOutput out) throws IOException {
     assert block >= 0 && block < 65536;
@@ -101,49 +98,19 @@ final class IndexedDISI extends DocIdSetIterator {
   /** The slice that stores the {@link DocIdSetIterator}. */
   private final IndexInput slice;
   private final long cost;
-  private final IndexedDISICache cache;
 
   IndexedDISI(IndexInput in, long offset, long length, long cost) throws IOException {
-    this(in, offset, length, cost, NO_NAME);
-  }
-
-  IndexedDISI(IndexInput in, long offset, long length, long cost, String name) throws IOException {
-    this(in, offset, length, cost, null, name);
-  }
-
-  IndexedDISI(IndexInput in, long offset, long length, long cost, IndexedDISICache cache) throws IOException {
-    this(in, offset, length, cost, cache, NO_NAME);
-  }
-
-  IndexedDISI(IndexInput in, long offset, long length, long cost, IndexedDISICache cache, String name) throws IOException {
-    this(in.slice("docs", offset, length), cost, cache, name);
-  }
-
-  IndexedDISI(IndexInput slice, long cost) throws IOException {
-    this(slice, cost, NO_NAME);
-  }
-  // This constructor allows to pass the slice directly in case it helps reuse
-  // see eg. Lucene70 norms producer's merge instance
-  IndexedDISI(IndexInput slice, long cost, String name) throws IOException {
-    this(slice, cost, null, name);
-//    IndexedDISICacheFactory.debug(
-//        "Non-cached direct slice IndexedDISI with length " + slice.length() + ": " + slice.toString());
+    this(in.slice("docs", offset, length), cost);
   }
 
-  IndexedDISI(IndexInput slice, long cost, IndexedDISICache cache) throws IOException {
-    this(slice, cost, cache, NO_NAME);
-  }
   // This constructor allows to pass the slice directly in case it helps reuse
   // see eg. Lucene70 norms producer's merge instance
-  IndexedDISI(IndexInput slice, long cost, IndexedDISICache cache, String name) {
-    this.name = name;
+  IndexedDISI(IndexInput slice, long cost) throws IOException {
     this.slice = slice;
     this.cost = cost;
-    this.cache = cache == null ? IndexedDISICache.EMPTY : cache;
   }
 
   private int block = -1;
-  private long blockStart; // Used with the DENSE cache
   private long blockEnd;
   private int nextBlockIndex = -1;
   Method method;
@@ -159,8 +126,6 @@ final class IndexedDISI extends DocIdSetIterator {
   private int wordIndex = -1;
   // number of one bits encountered so far, including those of `word`
   private int numberOfOnes;
-  // Used with rank for jumps inside of DENSE
-  private int denseOrigoIndex;
 
   // ALL variables
   private int gap;
@@ -173,7 +138,6 @@ final class IndexedDISI extends DocIdSetIterator {
   @Override
   public int advance(int target) throws IOException {
     final int targetBlock = target & 0xFFFF0000;
-    // Note: The cache makes it easy to add support for random access. This has not been done as the API forbids it
     if (block < targetBlock) {
       advanceBlock(targetBlock);
     }
@@ -199,20 +163,6 @@ final class IndexedDISI extends DocIdSetIterator {
   }
 
   private void advanceBlock(int targetBlock) throws IOException {
-    if (targetBlock >= block+2) { // 1 block skip is (slightly) faster to do without block jump table
-      long offset = cache.getFilePointerForBlock(targetBlock >> IndexedDISICache.BLOCK_BITS);
-      if (offset != -1 && offset > slice.getFilePointer()) {
-        int origo = cache.getIndexForBlock(targetBlock >> IndexedDISICache.BLOCK_BITS);
-        if (origo != -1) {
-          this.nextBlockIndex = origo - 1; // -1 to compensate for the always-added 1 in readBlockHeader
-          slice.seek(offset);
-          readBlockHeader();
-          return;
-        }
-      }
-    }
-
-    // Fallback to non-cached
     do {
       slice.seek(blockEnd);
       readBlockHeader();
@@ -220,7 +170,6 @@ final class IndexedDISI extends DocIdSetIterator {
   }
 
   private void readBlockHeader() throws IOException {
-    blockStart = slice.getFilePointer();
     block = Short.toUnsignedInt(slice.readShort()) << 16;
     assert block >= 0;
     final int numValues = 1 + Short.toUnsignedInt(slice.readShort());
@@ -238,7 +187,6 @@ final class IndexedDISI extends DocIdSetIterator {
       blockEnd = slice.getFilePointer() + (1 << 13);
       wordIndex = -1;
       numberOfOnes = index + 1;
-      denseOrigoIndex = numberOfOnes;
     }
   }
 
@@ -302,7 +250,6 @@ final class IndexedDISI extends DocIdSetIterator {
       boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException {
         final int targetInBlock = target & 0xFFFF;
         final int targetWordIndex = targetInBlock >>> 6;
-
         for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) {
           disi.word = disi.slice.readLong();
           disi.numberOfOnes += Long.bitCount(disi.word);
@@ -316,10 +263,7 @@ final class IndexedDISI extends DocIdSetIterator {
           return true;
         }
 
-        // There were no set bits at the wanted position. Move forward until one is reached
         while (++disi.wordIndex < 1024) {
-          // This could use the rank cache to skip empty spaces >= 512 bits, but it seems unrealistic
-          // that such blocks would be DENSE
           disi.word = disi.slice.readLong();
           if (disi.word != 0) {
             disi.index = disi.numberOfOnes;
@@ -328,15 +272,12 @@ final class IndexedDISI extends DocIdSetIterator {
             return true;
           }
         }
-        // No set bits in the block at or after the wanted position.
         return false;
       }
-
       @Override
       boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException {
         final int targetInBlock = target & 0xFFFF;
         final int targetWordIndex = targetInBlock >>> 6;
-
         for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) {
           disi.word = disi.slice.readLong();
           disi.numberOfOnes += Long.bitCount(disi.word);
@@ -347,8 +288,6 @@ final class IndexedDISI extends DocIdSetIterator {
         disi.index = disi.numberOfOnes - Long.bitCount(leftBits);
         return (leftBits & 1L) != 0;
       }
-
-
     },
     ALL {
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a20705b/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
deleted file mode 100644
index 9a80689..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICache.java
+++ /dev/null
@@ -1,234 +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.io.IOException;
-import java.util.Arrays;
-import java.util.Locale;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.PackedInts;
-
-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.
- *
- * 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
- * one block at a time.
- *
- * Each long entry consists of 2 logical parts:
- *
- * The first 31 bits holds the index (number of set bits in the blocks) up to just before the
- * wanted block. The next 33 bits holds the offset into the underlying slice.
- * As there is a maximum of 2^16 blocks, it follows that the maximum size of any block must
- * not exceed 2^17 bits to avoid overflow. This is currently the case, with the largest
- * block being DENSE and using 2^16 + 32 bits, and is likely to continue to hold as using
- * more than double the amount of bits is unlikely to be an efficient representation.
- * The cache overhead is numDocs/1024 bytes.
- *
- * Note: There are 4 types of blocks: ALL, DENSE, SPARSE and non-existing (0 set bits).
- * 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 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
-  static final int BLOCK_BITS = 16;
-  private static final long BLOCK_INDEX_SHIFT = 33; // Number of bits to shift a lookup entry to get the index
-  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 long[] blockCache = null; // One every 65536 docs, contains index & slice position
-  private String creationStats = "";
-  private final String name; // Identifier for debug, log & inspection
-
-  // Flags for not-yet-defined-values used during building
-  private static final long BLOCK_EMPTY_INDEX = ~0L << BLOCK_INDEX_SHIFT;
-  private static final long BLOCK_EMPTY_LOOKUP = BLOCK_LOOKUP_MASK;
-  private static final long BLOCK_EMPTY = BLOCK_EMPTY_INDEX | BLOCK_EMPTY_LOOKUP;
-
-  /**
-   * Builds the stated caches for the given IndexInput.
-   *
-   * @param in positioned at the start of the logical underlying bitmap.
-   */
-  IndexedDISICache(IndexInput in, String name) throws IOException {
-    blockCache = new long[16];    // Will be extended when needed
-    Arrays.fill(blockCache, BLOCK_EMPTY);
-    this.name = name;
-    updateCaches(in);
-  }
-
-  private IndexedDISICache() {
-    this.blockCache = null;
-    this.name = "";
-  }
-
-  // Used to represent no caching.
-  public static final IndexedDISICache EMPTY = new IndexedDISICache();
-
-  /**
-   * If available, returns a position within the underlying {@link IndexInput} for the start of the block
-   * containing the wanted bit (the target) or the next non-EMPTY block, if the block representing the bit is empty.
-   * @param targetBlock the index for the block to resolve (docID / 65536).
-   * @return the offset for the block for target or -1 if it cannot be resolved.
-   */
-  long getFilePointerForBlock(int targetBlock) {
-    long offset = blockCache == null || blockCache.length <= targetBlock ?
-        -1 : blockCache[targetBlock] & BLOCK_LOOKUP_MASK;
-    return offset == BLOCK_EMPTY_LOOKUP ? -1 : offset;
-  }
-
-  /**
-   * If available, returns the index; number of set bits before the wanted block.
-   * @param targetBlock the block to resolve (docID / 65536).
-   * @return the index for the block or -1 if it cannot be resolved.
-   */
-  int getIndexForBlock(int targetBlock) {
-    if (blockCache == null || blockCache.length <= targetBlock) {
-      return -1;
-    }
-    return (blockCache[targetBlock] & BLOCK_INDEX_MASK) == BLOCK_EMPTY_INDEX ?
-        -1 : (int)(blockCache[targetBlock] >>> BLOCK_INDEX_SHIFT);
-  }
-
-  public boolean hasOffsets() {
-    return blockCache != null;
-  }
-
-  private void updateCaches(IndexInput slice) throws IOException {
-    final long startOffset = slice.getFilePointer();
-
-    final long startTime = System.nanoTime();
-    AtomicInteger statBlockALL = new AtomicInteger(0);
-    AtomicInteger statBlockDENSE = new AtomicInteger(0);
-    AtomicInteger statBlockSPARSE = new AtomicInteger(0);
-
-    // Fill phase
-    int largestBlock = fillCache(slice, statBlockALL, statBlockDENSE, statBlockSPARSE);
-    freezeCaches(largestBlock);
-
-    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",
-        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);
-  }
-
-  private int fillCache(
-      IndexInput slice, AtomicInteger statBlockALL, AtomicInteger statBlockDENSE, AtomicInteger statBlockSPARSE)
-      throws IOException {
-    int largestBlock = -1;
-    long index = 0;
-    int rankIndex = -1;
-    while (slice.getFilePointer() < slice.length()) {
-      final long startFilePointer = slice.getFilePointer();
-
-      final int blockIndex = Short.toUnsignedInt(slice.readShort());
-      final int numValues = 1 + Short.toUnsignedInt(slice.readShort());
-
-      assert blockIndex > largestBlock;
-      if (blockIndex == DocIdSetIterator.NO_MORE_DOCS >>> 16) { // End reached
-        assert Short.toUnsignedInt(slice.readShort()) == (DocIdSetIterator.NO_MORE_DOCS & 0xFFFF);
-        break;
-      }
-      largestBlock = blockIndex;
-
-      blockCache = ArrayUtil.grow(blockCache, blockIndex+1); // No-op if large enough
-      blockCache[blockIndex] = (index << BLOCK_INDEX_SHIFT) | startFilePointer;
-      index += numValues;
-
-      if (numValues <= MAX_ARRAY_LENGTH) { // SPARSE
-        statBlockSPARSE.incrementAndGet();
-        slice.seek(slice.getFilePointer() + (numValues << 1));
-        continue;
-      }
-      if (numValues == 65536) { // ALL
-        statBlockALL.incrementAndGet();
-        // Already at next block offset
-        continue;
-      }
-
-      // The block is DENSE
-      statBlockDENSE.incrementAndGet();
-      long nextBlockOffset = slice.getFilePointer() + (1 << 13);
-      slice.seek(nextBlockOffset);
-    }
-
-    return largestBlock;
-  }
-
-  private void freezeCaches(int largestBlock) {
-    if (largestBlock == -1) { // No set bit: Disable the caches
-      blockCache = null;
-      return;
-    }
-
-    // Reduce size to minimum
-    if (blockCache.length-1 > largestBlock) {
-      long[] newBC = new long[Math.max(largestBlock - 1, 1)];
-      System.arraycopy(blockCache, 0, newBC, 0, newBC.length);
-      blockCache = newBC;
-    }
-
-    // Set non-defined blockCache entries (caused by blocks with 0 set bits) to the subsequently defined one
-    long latest = BLOCK_EMPTY;
-    for (int i = blockCache.length-1; i >= 0 ; i--) {
-      long current = blockCache[i];
-      if (current == BLOCK_EMPTY) {
-        blockCache[i] = latest;
-      } else {
-        latest = current;
-      }
-    }
-  }
-
-  /**
-   * @return Human readable details from the creation of the cache instance.
-   */
-  public String getCreationStats() {
-    return creationStats;
-  }
-
-  /**
-   * @return Human-readable name for the cache instance.
-   */
-  public String getName() {
-    return name;
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return (blockCache == null ? 0 : RamUsageEstimator.sizeOf(blockCache)) +
-        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/8a20705b/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
deleted file mode 100644
index 6cf8af1..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/IndexedDISICacheFactory.java
+++ /dev/null
@@ -1,150 +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.io.IOException;
-import java.util.HashMap;
-import java.util.Locale;
-import java.util.Map;
-
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RandomAccessInput;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-
-/**
- * Creates and stores caches for {@link IndexedDISI} and {@link Lucene70DocValuesProducer}.
- * The caches are stored in maps, where the key is made up from offset and length of a slice
- * in an underlying segment. Each segment uses their own IndexedDISICacheFactory.
- *
- * See {@link IndexedDISICache} for details on the caching.
- */
-public class IndexedDISICacheFactory implements Accountable {
-
-  /**
-   * If the slice with the DISI-data is less than this number of bytes, don't create a cache.
-   * This is a very low number as the DISI-structure very efficiently represents EMPTY and ALL blocks.
-   */
-  private static int MIN_LENGTH_FOR_CACHING = 50; // Set this very low: Could be 9 EMPTY followed by a SPARSE
-
-  // jump-table and rank for DISI blocks
-  private final Map<Long, IndexedDISICache> disiPool = new HashMap<>();
-
-  /**
-   * Create a cached {@link IndexedDISI} instance.
-   * @param data   persistent data containing the DISI-structure.
-   * @param cost   cost as defined for IndexedDISI.
-   * @param name   identifier for the DISI-structure for debug purposes.
-   * @return a cached IndexedDISI or a plain IndexedDISI, if caching is not applicable.
-   * @throws IOException if the DISI-structure could not be accessed.
-   */
-  IndexedDISI createCachedIndexedDISI(IndexInput data, long key, int cost, String name) throws IOException {
-    IndexedDISICache cache = getCache(data, key, name);
-    return new IndexedDISI(data, cost, cache, name);
-  }
-
-  /**
-   * Create a cached {@link IndexedDISI} instance.
-   * @param data   persistent data containing the DISI-structure.
-   * @param offset same as the offset that will also be used for creating an {@link IndexedDISI}.
-   * @param length same af the length that will also be used for creating an {@link IndexedDISI}.
-   * @param cost   cost as defined for IndexedDISI.
-   * @param name   identifier for the DISI-structure for debug purposes.
-   * @return a cached IndexedDISI or a plain IndexedDISI, if caching is not applicable.
-   * @throws IOException if the DISI-structure could not be accessed.
-   */
-  IndexedDISI createCachedIndexedDISI(IndexInput data, long offset, long length, long cost, String name)
-      throws IOException {
-    IndexedDISICache cache = getCache(data, offset, length, name);
-    return new IndexedDISI(data, offset, length, cost, cache, name);
-  }
-
-  /**
-   * Creates a cache (jump table) for {@link IndexedDISI}.
-   * If the cache has previously been created, the old cache is returned.
-   * @param data   the slice to create a cache for.
-   * @param offset same as the offset that will also be used for creating an {@link IndexedDISI}.
-   * @param length same af the length that will also be used for creating an {@link IndexedDISI}.
-   * @param name human readable designation, typically a field name. Used for debug, log and inspection.
-   * @return a cache for the given slice+offset+length or null if not suitable for caching.
-   */
-  public IndexedDISICache getCache(IndexInput data, long offset, long length, String name) throws IOException {
-    if (length < MIN_LENGTH_FOR_CACHING) {
-      return null;
-    }
-
-    long key = offset + length;
-    IndexedDISICache cache = disiPool.get(key);
-    if (cache == null) {
-      // TODO: Avoid overlapping builds of the same cache for performance reason
-      cache = new IndexedDISICache(data.slice("docs", offset, length), name);
-      disiPool.put(key, cache);
-    }
-    return cache;
-  }
-
-  /**
-   * Creates a cache (jump table) for {@link IndexedDISI}.
-   * If the cache has previously been created, the old cache is returned.
-   * @param slice the input slice.
-   * @param key identifier for the cache, unique within the segment that originated the slice.
-   *            Recommendation is offset+length for the slice, relative to the data mapping the segment.
-   *            Warning: Do not use slice.getFilePointer and slice.length as they are not guaranteed
-   *            to be unique within the segment (slice.getFilePointer is 0 when a sub-slice is created).
-   * @param name human readable designation, typically a field name. Used for debug, log and inspection.
-   * @return a cache for the given slice+offset+length or null if not suitable for caching.
-   */
-  public IndexedDISICache getCache(IndexInput slice, long key, String name) throws IOException {
-    final long length = slice.length();
-    if (length < MIN_LENGTH_FOR_CACHING) {
-      return null;
-    }
-
-    IndexedDISICache cache = disiPool.get(key);
-    if (cache == null) {
-      // TODO: Avoid overlapping builds of the same cache
-      cache = new IndexedDISICache(slice, name);
-      disiPool.put(key, cache);
-    }
-    return cache;
-  }
-
-  // Statistics
-  public long getDISIBlocksWithOffsetsCount() {
-    return disiPool.values().stream().filter(IndexedDISICache::hasOffsets).count();
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    long mem = RamUsageEstimator.shallowSizeOf(this) +
-        RamUsageEstimator.shallowSizeOf(disiPool);
-    for (Map.Entry<Long, IndexedDISICache> cacheEntry: disiPool.entrySet()) {
-      mem += RamUsageEstimator.shallowSizeOf(cacheEntry);
-      mem += RamUsageEstimator.sizeOf(cacheEntry.getKey());
-      mem += cacheEntry.getValue().ramBytesUsed();
-    }
-    return mem;
-  }
-
-  /**
-   * Releases all caches.
-   */
-  void releaseAll() {
-    disiPool.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a20705b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
index 812caba..b0f6e84 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
@@ -57,7 +57,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
   private final Map<String,SortedNumericEntry> sortedNumerics = new HashMap<>();
   private long ramBytesUsed;
   private final IndexInput data;
-  private final IndexedDISICacheFactory disiCacheFactory = new IndexedDISICacheFactory();
   private final int maxDoc;
 
   /** expert: instantiates a new reader */
@@ -120,23 +119,23 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       }
       byte type = meta.readByte();
       if (type == Lucene70DocValuesFormat.NUMERIC) {
-        numerics.put(info.name, readNumeric(meta, info.name));
+        numerics.put(info.name, readNumeric(meta));
       } else if (type == Lucene70DocValuesFormat.BINARY) {
-        binaries.put(info.name, readBinary(meta, info.name));
+        binaries.put(info.name, readBinary(meta));
       } else if (type == Lucene70DocValuesFormat.SORTED) {
-        sorted.put(info.name, readSorted(meta, info.name));
+        sorted.put(info.name, readSorted(meta));
       } else if (type == Lucene70DocValuesFormat.SORTED_SET) {
-        sortedSets.put(info.name, readSortedSet(meta, info.name));
+        sortedSets.put(info.name, readSortedSet(meta));
       } else if (type == Lucene70DocValuesFormat.SORTED_NUMERIC) {
-        sortedNumerics.put(info.name, readSortedNumeric(meta, info.name));
+        sortedNumerics.put(info.name, readSortedNumeric(meta));
       } else {
         throw new CorruptIndexException("invalid type: " + type, meta);
       }
     }
   }
 
-  private NumericEntry readNumeric(ChecksumIndexInput meta, String name) throws IOException {
-    NumericEntry entry = new NumericEntry(name);
+  private NumericEntry readNumeric(ChecksumIndexInput meta) throws IOException {
+    NumericEntry entry = new NumericEntry();
     readNumeric(meta, entry);
     return entry;
   }
@@ -168,8 +167,8 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     entry.valuesLength = meta.readLong();
   }
 
-  private BinaryEntry readBinary(ChecksumIndexInput meta, String name) throws IOException {
-    BinaryEntry entry = new BinaryEntry(name);
+  private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
+    BinaryEntry entry = new BinaryEntry();
     entry.dataOffset = meta.readLong();
     entry.dataLength = meta.readLong();
     entry.docsWithFieldOffset = meta.readLong();
@@ -187,8 +186,8 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     return entry;
   }
 
-  private SortedEntry readSorted(ChecksumIndexInput meta, String name) throws IOException {
-    SortedEntry entry = new SortedEntry(name);
+  private SortedEntry readSorted(ChecksumIndexInput meta) throws IOException {
+    SortedEntry entry = new SortedEntry();
     entry.docsWithFieldOffset = meta.readLong();
     entry.docsWithFieldLength = meta.readLong();
     entry.numDocsWithField = meta.readInt();
@@ -199,12 +198,12 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     return entry;
   }
 
-  private SortedSetEntry readSortedSet(ChecksumIndexInput meta, String name) throws IOException {
-    SortedSetEntry entry = new SortedSetEntry(name);
+  private SortedSetEntry readSortedSet(ChecksumIndexInput meta) throws IOException {
+    SortedSetEntry entry = new SortedSetEntry();
     byte multiValued = meta.readByte();
     switch (multiValued) {
       case 0: // singlevalued
-        entry.singleValueEntry = readSorted(meta, name);
+        entry.singleValueEntry = readSorted(meta);
         return entry;
       case 1: // multivalued
         break;
@@ -246,8 +245,8 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     entry.termsIndexAddressesLength = meta.readLong();
   }
 
-  private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta, String name) throws IOException {
-    SortedNumericEntry entry = new SortedNumericEntry(name);
+  private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta) throws IOException {
+    SortedNumericEntry entry = new SortedNumericEntry();
     readNumeric(meta, entry);
     entry.numDocsWithField = meta.readInt();
     if (entry.numDocsWithField != entry.numValues) {
@@ -263,23 +262,9 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
   @Override
   public void close() throws IOException {
     data.close();
-    disiCacheFactory.releaseAll();
   }
 
-  // Highly debatable if this is a sane construct as the name is only used for debug/logging/inspection purposes
-  // This was introduced in LUCENE-8374
-  private static class EntryImpl {
-    final String name;
-
-    public EntryImpl(String name) {
-      this.name = name;
-    }
-  }
-
-  private static class NumericEntry extends EntryImpl {
-    public NumericEntry(String name) {
-      super(name);
-    }
+  private static class NumericEntry {
     long[] table;
     int blockShift;
     byte bitsPerValue;
@@ -292,10 +277,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     long valuesLength;
   }
 
-  private static class BinaryEntry extends EntryImpl {
-    public BinaryEntry(String name) {
-      super(name);
-    }
+  private static class BinaryEntry {
     long dataOffset;
     long dataLength;
     long docsWithFieldOffset;
@@ -308,10 +290,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     DirectMonotonicReader.Meta addressesMeta;
   }
 
-  private static class TermsDictEntry extends EntryImpl {
-    public TermsDictEntry(String name) {
-      super(name);
-    }
+  private static class TermsDictEntry {
     long termsDictSize;
     int termsDictBlockShift;
     DirectMonotonicReader.Meta termsAddressesMeta;
@@ -329,9 +308,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
   }
 
   private static class SortedEntry extends TermsDictEntry {
-    public SortedEntry(String name) {
-      super(name);
-    }
     long docsWithFieldOffset;
     long docsWithFieldLength;
     int numDocsWithField;
@@ -341,9 +317,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
   }
 
   private static class SortedSetEntry extends TermsDictEntry {
-    public SortedSetEntry(String name) {
-      super(name);
-    }
     SortedEntry singleValueEntry;
     long docsWithFieldOffset;
     long docsWithFieldLength;
@@ -357,9 +330,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
   }
 
   private static class SortedNumericEntry extends NumericEntry {
-    public SortedNumericEntry(String name) {
-      super(name);
-    }
     int numDocsWithField;
     DirectMonotonicReader.Meta addressesMeta;
     long addressesOffset;
@@ -368,7 +338,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
 
   @Override
   public long ramBytesUsed() {
-    return ramBytesUsed + disiCacheFactory.ramBytesUsed();
+    return ramBytesUsed;
   }
 
   @Override
@@ -526,8 +496,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       }
     } else {
       // sparse
-      final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
-          data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numValues, entry.name);
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numValues);
       if (entry.bitsPerValue == 0) {
         return new SparseNumericDocValues(disi) {
           @Override
@@ -798,8 +767,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       }
     } else {
       // sparse
-      final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
-          data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
       if (entry.minLength == entry.maxLength) {
         // fixed length
         final int length = entry.maxLength;
@@ -900,8 +868,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       };
     } else {
       // sparse
-      final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
-          data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
       return new BaseSortedDocValues(entry, data) {
 
         @Override
@@ -1269,8 +1236,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       };
     } else {
       // sparse
-      final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
-          data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
       return new SortedNumericDocValues() {
 
         boolean set;
@@ -1396,8 +1362,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       };
     } else {
       // sparse
-      final IndexedDISI disi = disiCacheFactory.createCachedIndexedDISI(
-          data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField, entry.name);
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
       return new BaseSortedSetDocValues(entry, data) {
 
         boolean set;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a20705b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java
index aae3a7f..64bfbd5 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestIndexedDISI.java
@@ -150,39 +150,6 @@ public class TestIndexedDISI extends LuceneTestCase {
       }
     }
   }
-  public void testDenseMultiBlock() throws IOException {
-    try (Directory dir = newDirectory()) {
-      int maxDoc = 10 * 65536; // 10 blocks
-      FixedBitSet set = new FixedBitSet(maxDoc);
-      for (int i = 0; i < maxDoc; i += 2) { // Set every other to ensure dense
-        set.set(i);
-      }
-      doTest(set, dir);
-    }
-  }
-
-  public void testOneDocMissingFixed() throws IOException {
-    int maxDoc = 9699;
-    FixedBitSet set = new FixedBitSet(maxDoc);
-    set.set(0, maxDoc);
-    set.clear(1345);
-    try (Directory dir = newDirectory()) {
-
-      final int cardinality = set.cardinality();
-      long length;
-      try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) {
-        IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out);
-        length = out.getFilePointer();
-      }
-
-      int step = 16000;
-      try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
-        IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
-        BitSetIterator disi2 = new BitSetIterator(set, cardinality);
-        assertAdvanceEquality(disi, disi2, step);
-      }
-    }
-  }
 
   public void testRandom() throws IOException {
     try (Directory dir = newDirectory()) {
@@ -221,14 +188,32 @@ public class TestIndexedDISI extends LuceneTestCase {
     try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
       IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
       BitSetIterator disi2 = new BitSetIterator(set, cardinality);
-      assertSingleStepEquality(disi, disi2);
+      int i = 0;
+      for (int doc = disi2.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = disi2.nextDoc()) {
+        assertEquals(doc, disi.nextDoc());
+        assertEquals(i++, disi.index());
+      }
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, disi.nextDoc());
     }
 
     for (int step : new int[] {1, 10, 100, 1000, 10000, 100000}) {
       try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
         IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
         BitSetIterator disi2 = new BitSetIterator(set, cardinality);
-        assertAdvanceEquality(disi, disi2, step);
+        int index = -1;
+        while (true) {
+          int target = disi2.docID() + step;
+          int doc;
+          do {
+            doc = disi2.nextDoc();
+            index++;
+          } while (doc < target);
+          assertEquals(doc, disi.advance(target));
+          if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+            break;
+          }
+          assertEquals(index, disi.index());
+        }
       }
     }
 
@@ -236,18 +221,8 @@ public class TestIndexedDISI extends LuceneTestCase {
       try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
         IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
         BitSetIterator disi2 = new BitSetIterator(set, cardinality);
-        int disi2length = set.length();
-        assertAdvanceExactRandomized(disi, disi2, disi2length, step);
-      }
-    }
-
-    dir.deleteFile("foo");
-  }
-
-  private void assertAdvanceExactRandomized(IndexedDISI disi, BitSetIterator disi2, int disi2length, int step)
-      throws IOException {
         int index = -1;
-    for (int target = 0; target < disi2length; ) {
+        for (int target = 0; target < set.length(); ) {
           target += TestUtil.nextInt(random(), 0, step);
           int doc = disi2.docID();
           while (doc < target) {
@@ -266,31 +241,9 @@ public class TestIndexedDISI extends LuceneTestCase {
           }
         }
       }
-
-  private void assertSingleStepEquality(IndexedDISI disi, BitSetIterator disi2) throws IOException {
-    int i = 0;
-    for (int doc = disi2.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = disi2.nextDoc()) {
-      assertEquals(doc, disi.nextDoc());
-      assertEquals(i++, disi.index());
-    }
-    assertEquals(DocIdSetIterator.NO_MORE_DOCS, disi.nextDoc());
     }
 
-  private void assertAdvanceEquality(IndexedDISI disi, BitSetIterator disi2, int step) throws IOException {
-    int index = -1;
-    while (true) {
-      int target = disi2.docID() + step;
-      int doc;
-      do {
-        doc = disi2.nextDoc();
-        index++;
-      } while (doc < target);
-      assertEquals(doc, disi.advance(target));
-      if (doc == DocIdSetIterator.NO_MORE_DOCS) {
-        break;
-      }
-      assertEquals("Expected equality using step " + step + " at docID " + doc, index, disi.index());
-    }
+    dir.deleteFile("foo");
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8a20705b/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java b/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
index ebc045a..0214e54 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
@@ -18,13 +18,7 @@ package org.apache.lucene.index;
 
 
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.List;
 
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -33,9 +27,7 @@ import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
-import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MMapDirectory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -131,103 +123,6 @@ public class TestDocValues extends LuceneTestCase {
     iw.close();
     dir.close();
   }
-
-  /**
-   * Triggers varying bits per value codec representation for numeric.
-   */
-  public void testNumericFieldVaryingBPV() throws Exception {
-    Directory dir = newDirectory();
-    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
-    long generatedSum = 0;
-    for (int bpv = 2 ; bpv < 24 ; bpv+=3) {
-      for (int i = 0 ; i < 66000 ; i++) {
-        Document doc = new Document();
-        int max = 1 << (bpv - 1);
-        int value =  random().nextInt(max) | max;
-        generatedSum += value;
-        //System.out.println("--- " + value);
-        doc.add(new NumericDocValuesField("foo", value));
-        iw.addDocument(doc);
-      }
-    }
-    iw.flush();
-    iw.forceMerge(1, true);
-    iw.commit();
-    DirectoryReader dr = DirectoryReader.open(iw);
-    LeafReader r = getOnlyLeafReader(dr);
-
-    // ok
-    NumericDocValues numDV = DocValues.getNumeric(r, "foo");
-
-    assertNotNull(numDV);
-    long sum = 0;
-    while (numDV.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-      sum += numDV.longValue();
-    }
-    assertEquals("The sum of retrieved values should match the input", generatedSum, sum);
-
-//    assertNotNull(DocValues.getSortedNumeric(r, "foo"));
-
-    dr.close();
-    iw.close();
-    dir.close();
-  }
-
-  // LUCENE-8374 had a bug where a vBPV-block with BPV==0 as the very end of the numeric DocValues made it fail
-  public void testNumericEntryZeroesLastBlock() throws IOException {
-    List<Long> docValues = new ArrayList<>(2*16384);
-    for (int id = 0 ; id < 2*16384 ; id++) { // 2 vBPV-blocks for the dv-field
-      if (id < 16384) { // First vBPV-block just has semi-ramdom values
-        docValues.add((long) (id % 1000));
-      } else {          // Second block is all zeroes, resulting in an extreme "1-byte for the while block"-representation
-        docValues.add(0L);
-      }
-    }
-    assertRandomAccessDV("Last block BPV=0", docValues);
-  }
-
-  private void assertRandomAccessDV(String designation, List<Long> docValues) throws IOException {
-    // Create corpus
-    Path zeroPath = Paths.get(System.getProperty("java.io.tmpdir"),"plain_" + random().nextInt());
-    Directory zeroDir = new MMapDirectory(zeroPath);
-    IndexWriterConfig iwc = new IndexWriterConfig(new StandardAnalyzer());
-    //iwc.setCodec(Codec.forName("Lucene70"));
-    IndexWriter iw = new IndexWriter(zeroDir, iwc);
-
-    for (int id = 0 ; id < docValues.size() ; id++) {
-      Document doc = new Document();
-      doc.add(new StringField("id", Integer.toString(id), Field.Store.YES));
-      doc.add(new NumericDocValuesField("dv", docValues.get(id)));
-      iw.addDocument(doc);
-    }
-    iw.flush();
-    iw.commit();
-    iw.forceMerge(1, true);
-    iw.close();
-
-    DirectoryReader dr = DirectoryReader.open(zeroDir);
-    for (int id = 0 ; id < docValues.size() ; id++) {
-      int readerIndex = dr.readerIndex(id);
-      // We create a new reader each time as we want to test vBPV-skipping and not sequential iteration
-      NumericDocValues numDV = dr.leaves().get(readerIndex).reader().getNumericDocValues("dv");
-      assertTrue(designation + ": There should be a value for docID " + id, numDV.advanceExact(id));
-      assertEquals(designation + ": The value for docID " + id + " should be as expected",
-          docValues.get(id), Long.valueOf(numDV.longValue()));
-    }
-    dr.close();
-
-    // Clean up
-    deleteAndClose(zeroDir);
-    Files.delete(zeroPath);
-  }
-
-  private void deleteAndClose(Directory dir) throws IOException {
-    String[] files = dir.listAll();
-    for (String file: files) {
-      dir.deleteFile(file);
-    }
-    dir.close();
-  }
   
   /** 
    * field with binary docvalues