You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by kr...@apache.org on 2016/10/20 19:31:18 UTC

[09/50] [abbrv] lucene-solr:jira/solr-8593: LUCENE-7489: Better sparsity support for Lucene70DocValuesFormat.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/927fd51d/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 e806ea5..755da79 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
@@ -18,84 +18,53 @@ package org.apache.lucene.codecs.lucene70;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene70.Lucene70DocValuesConsumer.NumberType;
-import org.apache.lucene.index.*;
-import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LongValues;
-import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.DirectMonotonicReader;
 import org.apache.lucene.util.packed.DirectReader;
-import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
-
-import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.*;
 
 /** reader for {@link Lucene70DocValuesFormat} */
 final class Lucene70DocValuesProducer extends DocValuesProducer implements Closeable {
   private final Map<String,NumericEntry> numerics = new HashMap<>();
   private final Map<String,BinaryEntry> binaries = new HashMap<>();
+  private final Map<String,SortedEntry> sorted = new HashMap<>();
   private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
-  private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
-  private final Map<String,NumericEntry> ords = new HashMap<>();
-  private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
-  private final int numFields;
-  private final AtomicLong ramBytesUsed;
+  private final Map<String,SortedNumericEntry> sortedNumerics = new HashMap<>();
+  private long ramBytesUsed;
   private final IndexInput data;
   private final int maxDoc;
 
-  // memory-resident structures
-  private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
-  private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
-  private final Map<String,DirectMonotonicReader.Meta> directAddressesMeta = new HashMap<>();
-
-  private final boolean merging;
-
-  // clone for merge: when merging we don't do any instances.put()s
-  Lucene70DocValuesProducer(Lucene70DocValuesProducer original) throws IOException {
-    assert Thread.holdsLock(original);
-    numerics.putAll(original.numerics);
-    binaries.putAll(original.binaries);
-    sortedSets.putAll(original.sortedSets);
-    sortedNumerics.putAll(original.sortedNumerics);
-    ords.putAll(original.ords);
-    ordIndexes.putAll(original.ordIndexes);
-    numFields = original.numFields;
-    ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
-    data = original.data.clone();
-    maxDoc = original.maxDoc;
-
-    addressInstances.putAll(original.addressInstances);
-    reverseIndexInstances.putAll(original.reverseIndexInstances);
-    merging = true;
-  }
-
   /** expert: instantiates a new reader */
   Lucene70DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     this.maxDoc = state.segmentInfo.maxDoc();
-    merging = false;
-    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+    ramBytesUsed = RamUsageEstimator.shallowSizeOfInstance(getClass());
 
     int version = -1;
-    int numFields = -1;
 
     // read in the entries from the metadata file.
     try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
@@ -106,7 +75,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
                                         Lucene70DocValuesFormat.VERSION_CURRENT,
                                         state.segmentInfo.getId(),
                                         state.segmentSuffix);
-        numFields = readFields(in, state.fieldInfos);
+        readFields(in, state.fieldInfos);
       } catch (Throwable exception) {
         priorE = exception;
       } finally {
@@ -114,7 +83,6 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       }
     }
 
-    this.numFields = numFields;
     String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
     this.data = state.directory.openInput(dataName, state.context);
     boolean success = false;
@@ -142,1277 +110,841 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     }
   }
 
-  private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
-    // sorted = binary + numeric
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene70DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n = readNumericEntry(info, meta);
-    ords.put(info.name, n);
-  }
-
-  private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
-    // sortedset = binary + numeric (addresses) + ordIndex
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene70DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n1 = readNumericEntry(info, meta);
-    ords.put(info.name, n1);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n2 = readNumericEntry(info, meta);
-    ordIndexes.put(info.name, n2);
-  }
-
-  private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
-    // sortedset table = binary + ordset table + ordset index
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene70DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n = readNumericEntry(info, meta);
-    ords.put(info.name, n);
-  }
-
-  private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
-    int numFields = 0;
-    int fieldNumber = meta.readVInt();
-    while (fieldNumber != -1) {
-      numFields++;
+  private void readFields(ChecksumIndexInput meta, FieldInfos infos) throws IOException {
+    for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
       FieldInfo info = infos.fieldInfo(fieldNumber);
       if (info == null) {
-        // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
         throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
       }
       byte type = meta.readByte();
       if (type == Lucene70DocValuesFormat.NUMERIC) {
-        numerics.put(info.name, readNumericEntry(info, meta));
+        numerics.put(info.name, readNumeric(meta));
       } else if (type == Lucene70DocValuesFormat.BINARY) {
-        BinaryEntry b = readBinaryEntry(info, meta);
-        binaries.put(info.name, b);
+        binaries.put(info.name, readBinary(meta));
       } else if (type == Lucene70DocValuesFormat.SORTED) {
-        readSortedField(info, meta);
+        sorted.put(info.name, readSorted(meta));
       } else if (type == Lucene70DocValuesFormat.SORTED_SET) {
-        SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedSets.put(info.name, ss);
-        if (ss.format == SORTED_WITH_ADDRESSES) {
-          readSortedSetFieldWithAddresses(info, meta);
-        } else if (ss.format == SORTED_SET_TABLE) {
-          readSortedSetFieldWithTable(info, meta);
-        } else if (ss.format == SORTED_SINGLE_VALUED) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene70DocValuesFormat.SORTED) {
-            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-          }
-          readSortedField(info, meta);
-        } else {
-          throw new AssertionError();
-        }
+        sortedSets.put(info.name, readSortedSet(meta));
       } else if (type == Lucene70DocValuesFormat.SORTED_NUMERIC) {
-        SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedNumerics.put(info.name, ss);
-        if (ss.format == SORTED_WITH_ADDRESSES) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          numerics.put(info.name, readNumericEntry(info, meta));
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          NumericEntry ordIndex = readNumericEntry(info, meta);
-          ordIndexes.put(info.name, ordIndex);
-        } else if (ss.format == SORTED_SET_TABLE) {
-          if (meta.readVInt() != info.number) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          NumericEntry n = readNumericEntry(info, meta);
-          ords.put(info.name, n);
-        } else if (ss.format == SORTED_SINGLE_VALUED) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene70DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          numerics.put(info.name, readNumericEntry(info, meta));
-        } else {
-          throw new AssertionError();
-        }
+        sortedNumerics.put(info.name, readSortedNumeric(meta));
       } else {
         throw new CorruptIndexException("invalid type: " + type, meta);
       }
-      fieldNumber = meta.readVInt();
     }
-    return numFields;
   }
 
-  private NumericEntry readNumericEntry(FieldInfo info, IndexInput meta) throws IOException {
+  private NumericEntry readNumeric(ChecksumIndexInput meta) throws IOException {
     NumericEntry entry = new NumericEntry();
-    entry.format = meta.readVInt();
-    entry.missingOffset = meta.readLong();
-    if (entry.format == SPARSE_COMPRESSED) {
-      // sparse bits need a bit more metadata
-      entry.numDocsWithValue = meta.readVLong();
-      final int blockShift = meta.readVInt();
-      entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithValue, blockShift);
-      ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
-      directAddressesMeta.put(info.name, entry.monotonicMeta);
+    readNumeric(meta, entry);
+    return entry;
+  }
+
+  private void readNumeric(ChecksumIndexInput meta, NumericEntry entry) throws IOException {
+    entry.docsWithFieldOffset = meta.readLong();
+    entry.docsWithFieldLength = meta.readLong();
+    entry.numValues = meta.readLong();
+    int tableSize = meta.readInt();
+    if (tableSize < -1 || tableSize > 256) {
+      throw new CorruptIndexException("invalid table size: " + tableSize, meta);
     }
-    entry.offset = meta.readLong();
-    entry.count = meta.readVLong();
-    switch(entry.format) {
-      case CONST_COMPRESSED:
-        entry.minValue = meta.readLong();
-        if (entry.count > Integer.MAX_VALUE) {
-          // currently just a limitation e.g. of bits interface and so on.
-          throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
-        }
-        break;
-      case GCD_COMPRESSED:
-        entry.minValue = meta.readLong();
-        entry.gcd = meta.readLong();
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case TABLE_COMPRESSED:
-        final int uniqueValues = meta.readVInt();
-        if (uniqueValues > 256) {
-          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
-        }
-        entry.table = new long[uniqueValues];
-        for (int i = 0; i < uniqueValues; ++i) {
-          entry.table[i] = meta.readLong();
-        }
-        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case DELTA_COMPRESSED:
-        entry.minValue = meta.readLong();
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case MONOTONIC_COMPRESSED:
-        final int blockShift = meta.readVInt();
-        entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, maxDoc + 1, blockShift);
-        ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
-        directAddressesMeta.put(info.name, entry.monotonicMeta);
-        break;
-      case SPARSE_COMPRESSED:
-        final byte numberType = meta.readByte();
-        switch (numberType) {
-          case 0:
-            entry.numberType = NumberType.VALUE;
-            break;
-          case 1:
-            entry.numberType = NumberType.ORDINAL;
-            break;
-          default:
-            throw new CorruptIndexException("Number type can only be 0 or 1, got=" + numberType, meta);
-        }
+    if (tableSize >= 0) {
+      entry.table = new long[tableSize];
+      ramBytesUsed += RamUsageEstimator.sizeOf(entry.table);
+      for (int i = 0; i < tableSize; ++i) {
+        entry.table[i] = meta.readLong();
+      }
+    }
+    entry.bitsPerValue = meta.readByte();
+    entry.minValue = meta.readLong();
+    entry.gcd = meta.readLong();
+    entry.valuesOffset = meta.readLong();
+    entry.valuesLength = meta.readLong();
+  }
 
-        // now read the numeric entry for non-missing values
-        final int fieldNumber = meta.readVInt();
-        if (fieldNumber != info.number) {
-          throw new CorruptIndexException("Field numbers mistmatch: " + fieldNumber + " != " + info.number, meta);
-        }
-        final int dvFormat = meta.readByte();
-        if (dvFormat != NUMERIC) {
-          throw new CorruptIndexException("Formats mistmatch: " + dvFormat + " != " + NUMERIC, meta);
-        }
-        entry.nonMissingValues = readNumericEntry(info, meta);
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
+  private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
+    BinaryEntry entry = new BinaryEntry();
+    entry.dataOffset = meta.readLong();
+    entry.dataLength = meta.readLong();
+    entry.docsWithFieldOffset = meta.readLong();
+    entry.docsWithFieldLength = meta.readLong();
+    entry.numDocsWithField = meta.readInt();
+    entry.minLength = meta.readInt();
+    entry.maxLength = meta.readInt();
+    if (entry.minLength < entry.maxLength) {
+      entry.addressesOffset = meta.readLong();
+      final int blockShift = meta.readVInt();
+      entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1L, blockShift);
+      ramBytesUsed += entry.addressesMeta.ramBytesUsed();
+      entry.addressesLength = meta.readLong();
     }
-    entry.endOffset = meta.readLong();
     return entry;
   }
 
-  private BinaryEntry readBinaryEntry(FieldInfo info, IndexInput meta) throws IOException {
-    BinaryEntry entry = new BinaryEntry();
-    entry.format = meta.readVInt();
-    entry.missingOffset = meta.readLong();
-    entry.minLength = meta.readVInt();
-    entry.maxLength = meta.readVInt();
-    entry.count = meta.readVLong();
-    entry.offset = meta.readLong();
-    switch(entry.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        break;
-      case BINARY_PREFIX_COMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        entry.packedIntsVersion = meta.readVInt();
-        entry.blockSize = meta.readVInt();
-        entry.reverseIndexOffset = meta.readLong();
-        break;
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        final int blockShift = meta.readVInt();
-        entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.count + 1, blockShift);
-        ramBytesUsed.addAndGet(entry.addressesMeta.ramBytesUsed());
-        directAddressesMeta.put(info.name, entry.addressesMeta);
-        entry.addressesEndOffset = meta.readLong();
+  private SortedEntry readSorted(ChecksumIndexInput meta) throws IOException {
+    SortedEntry entry = new SortedEntry();
+    entry.docsWithFieldOffset = meta.readLong();
+    entry.docsWithFieldLength = meta.readLong();
+    entry.numDocsWithField = meta.readInt();
+    entry.bitsPerValue = meta.readByte();
+    entry.ordsOffset = meta.readLong();
+    entry.ordsLength = meta.readLong();
+    readTermDict(meta, entry);
+    return entry;
+  }
+
+  private SortedSetEntry readSortedSet(ChecksumIndexInput meta) throws IOException {
+    SortedSetEntry entry = new SortedSetEntry();
+    byte multiValued = meta.readByte();
+    switch (multiValued) {
+      case 0: // singlevalued
+        entry.singleValueEntry = readSorted(meta);
+        return entry;
+      case 1: // multivalued
         break;
       default:
-        throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+        throw new CorruptIndexException("Invalid multiValued flag: " + multiValued, meta);
     }
+    entry.docsWithFieldOffset = meta.readLong();
+    entry.docsWithFieldLength = meta.readLong();
+    entry.bitsPerValue = meta.readByte();
+    entry.ordsOffset = meta.readLong();
+    entry.ordsLength = meta.readLong();
+    entry.numDocsWithField = meta.readInt();
+    entry.addressesOffset = meta.readLong();
+    final int blockShift = meta.readVInt();
+    entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift);
+    ramBytesUsed += entry.addressesMeta.ramBytesUsed();
+    entry.addressesLength = meta.readLong();
+    readTermDict(meta, entry);
     return entry;
   }
 
-  SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
-    SortedSetEntry entry = new SortedSetEntry();
-    entry.format = meta.readVInt();
-    if (entry.format == SORTED_SET_TABLE) {
-      final int totalTableLength = meta.readInt();
-      if (totalTableLength > 256) {
-        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
-      }
-      entry.table = new long[totalTableLength];
-      for (int i = 0; i < totalTableLength; ++i) {
-        entry.table[i] = meta.readLong();
-      }
-      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
-      final int tableSize = meta.readInt();
-      if (tableSize > totalTableLength + 1) { // +1 because of the empty set
-        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
-      }
-      entry.tableOffsets = new int[tableSize + 1];
-      for (int i = 1; i < entry.tableOffsets.length; ++i) {
-        entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
-      }
-      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
-    } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
-      throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+  private static void readTermDict(ChecksumIndexInput meta, TermsDictEntry entry) throws IOException {
+    entry.termsDictSize = meta.readVLong();
+    entry.termsDictBlockShift = meta.readInt();
+    final int blockShift = meta.readInt();
+    final long addressesSize = (entry.termsDictSize + (1L << entry.termsDictBlockShift) - 1) >>> entry.termsDictBlockShift;
+    entry.termsAddressesMeta = DirectMonotonicReader.loadMeta(meta, addressesSize, blockShift);
+    entry.maxTermLength = meta.readInt();
+    entry.termsDataOffset = meta.readLong();
+    entry.termsDataLength = meta.readLong();
+    entry.termsAddressesOffset = meta.readLong();
+    entry.termsAddressesLength = meta.readLong();
+    entry.termsDictIndexShift = meta.readInt();
+    final long indexSize = (entry.termsDictSize + (1L << entry.termsDictIndexShift) - 1) >>> entry.termsDictIndexShift;
+    entry.termsIndexAddressesMeta = DirectMonotonicReader.loadMeta(meta, 1 + indexSize, blockShift);
+    entry.termsIndexOffset = meta.readLong();
+    entry.termsIndexLength = meta.readLong();
+    entry.termsIndexAddressesOffset = meta.readLong();
+    entry.termsIndexAddressesLength = meta.readLong();
+  }
+
+  private SortedNumericEntry readSortedNumeric(ChecksumIndexInput meta) throws IOException {
+    SortedNumericEntry entry = new SortedNumericEntry();
+    readNumeric(meta, entry);
+    entry.numDocsWithField = meta.readInt();
+    if (entry.numDocsWithField != entry.numValues) {
+      entry.addressesOffset = meta.readLong();
+      final int blockShift = meta.readVInt();
+      entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithField + 1, blockShift);
+      ramBytesUsed += entry.addressesMeta.ramBytesUsed();
+      entry.addressesLength = meta.readLong();
     }
     return entry;
   }
 
   @Override
-  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.name);
-    DocIdSetIterator docsWithField;
-
-    if (entry.format == SPARSE_COMPRESSED) {
-      return getSparseNumericDocValues(entry);
-    } else {
-      if (entry.missingOffset == ALL_MISSING) {
-        return DocValues.emptyNumeric();
-      } else if (entry.missingOffset == ALL_LIVE) {
-        LongValues values = getNumeric(entry);
-        return new NumericDocValues() {
-          private int docID = -1;
-
-          @Override
-          public int docID() {
-            return docID;
-          }
-
-          @Override
-          public int nextDoc() {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-            }
-            return docID;
-          }
-
-          @Override
-          public int advance(int target) {
-            if (target >= maxDoc) {
-              docID = NO_MORE_DOCS;
-            } else {
-              docID = target;
-            }
-            return docID;
-          }
-
-          @Override
-          public long cost() {
-            return entry.count;
-          }
-
-          @Override
-          public long longValue() {
-            return values.get(docID);
-          }
-        };
-      } else {
-        docsWithField = getDocsWithField(entry.missingOffset, maxDoc);
-        LongValues values = getNumeric(entry);
-        return new NumericDocValues() {
-
-          @Override
-          public long longValue() {
-            return values.get(docsWithField.docID());
-          }
-
-          @Override
-          public int docID() {
-            return docsWithField.docID();
-          }
+  public void close() throws IOException {
+    data.close();
+  }
 
-          @Override
-          public int nextDoc() throws IOException {
-            return docsWithField.nextDoc();
-          }
+  private static class NumericEntry {
+    long[] table;
+    byte bitsPerValue;
+    long docsWithFieldOffset;
+    long docsWithFieldLength;
+    long numValues;
+    long minValue;
+    long gcd;
+    long valuesOffset;
+    long valuesLength;
+  }
 
-          @Override
-          public int advance(int target) throws IOException {
-            return docsWithField.advance(target);
-          }
+  private static class BinaryEntry {
+    long dataOffset;
+    long dataLength;
+    long docsWithFieldOffset;
+    long docsWithFieldLength;
+    int numDocsWithField;
+    int minLength;
+    int maxLength;
+    long addressesOffset;
+    long addressesLength;
+    DirectMonotonicReader.Meta addressesMeta;
+  }
 
-          @Override
-          public long cost() {
-            return docsWithField.cost();
-          }
+  private static class TermsDictEntry {
+    long termsDictSize;
+    int termsDictBlockShift;
+    DirectMonotonicReader.Meta termsAddressesMeta;
+    int maxTermLength;
+    long termsDataOffset;
+    long termsDataLength;
+    long termsAddressesOffset;
+    long termsAddressesLength;
+    int termsDictIndexShift;
+    DirectMonotonicReader.Meta termsIndexAddressesMeta;
+    long termsIndexOffset;
+    long termsIndexLength;
+    long termsIndexAddressesOffset;
+    long termsIndexAddressesLength;
+  }
 
-        };
-      }
-    }
+  private static class SortedEntry extends TermsDictEntry {
+    long docsWithFieldOffset;
+    long docsWithFieldLength;
+    int numDocsWithField;
+    byte bitsPerValue;
+    long ordsOffset;
+    long ordsLength;
   }
 
-  @Override
-  public long ramBytesUsed() {
-    return ramBytesUsed.get();
+  private static class SortedSetEntry extends TermsDictEntry {
+    SortedEntry singleValueEntry;
+    long docsWithFieldOffset;
+    long docsWithFieldLength;
+    int numDocsWithField;
+    byte bitsPerValue;
+    long ordsOffset;
+    long ordsLength;
+    DirectMonotonicReader.Meta addressesMeta;
+    long addressesOffset;
+    long addressesLength;
   }
 
-  @Override
-  public synchronized Collection<Accountable> getChildResources() {
-    List<Accountable> resources = new ArrayList<>();
-    resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
-    resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
-    resources.addAll(Accountables.namedAccountables("direct addresses meta field", directAddressesMeta));
-    return Collections.unmodifiableList(resources);
+  private static class SortedNumericEntry extends NumericEntry {
+    int numDocsWithField;
+    DirectMonotonicReader.Meta addressesMeta;
+    long addressesOffset;
+    long addressesLength;
   }
 
   @Override
-  public void checkIntegrity() throws IOException {
-    CodecUtil.checksumEntireFile(data);
+  public long ramBytesUsed() {
+    return ramBytesUsed;
   }
 
   @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(fields=" + numFields + ")";
+  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+    NumericEntry entry = numerics.get(field.name);
+    return getNumeric(entry);
   }
 
-  LongValues getNumeric(NumericEntry entry) throws IOException {
-    switch (entry.format) {
-      case CONST_COMPRESSED: {
-        final long constant = entry.minValue;
-        final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
-        return new LongValues() {
-          @Override
-          public long get(long index) {
-            return live.get((int)index) ? constant : 0;
-          }
-        };
-      }
-      case DELTA_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long delta = entry.minValue;
-        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return delta + values.get(id);
-          }
-        };
-      }
-      case GCD_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long min = entry.minValue;
-        final long mult = entry.gcd;
-        final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return min + mult * quotientReader.get(id);
-          }
-        };
-      }
-      case TABLE_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long table[] = entry.table;
-        final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return table[(int) ords.get(id)];
-          }
-        };
-      }
-      case SPARSE_COMPRESSED:
-        final SparseNumericDocValues values = getSparseNumericDocValues(entry);
-        final long missingValue;
-        switch (entry.numberType) {
-          case ORDINAL:
-            missingValue = -1L;
-            break;
-          case VALUE:
-            missingValue = 0L;
-            break;
-          default:
-            throw new AssertionError();
-        }
-        return new SparseNumericDocValuesRandomAccessWrapper(values, missingValue);
-      default:
-        throw new AssertionError();
-    }
-  }
+  private NumericDocValues getNumeric(NumericEntry entry) throws IOException {
+    if (entry.docsWithFieldOffset == -2) {
+      // empty
+      return DocValues.emptyNumeric();
+    } else if (entry.docsWithFieldOffset == -1) {
+      // dense
+      final LongValues normValues = getNumericValues(entry);
+      return new NumericDocValues() {
 
-  static final class SparseNumericDocValues extends NumericDocValues {
+        int doc = -1;
 
-    final int docIDsLength;
-    final LongValues docIds, values;
+        @Override
+        public long longValue() throws IOException {
+          return normValues.get(doc);
+        }
 
-    int index, doc;
+        @Override
+        public int docID() {
+          return doc;
+        }
 
-    SparseNumericDocValues(int docIDsLength, LongValues docIDs, LongValues values) {
-      this.docIDsLength = docIDsLength;
-      this.docIds = docIDs;
-      this.values = values;
-      reset();
-    }
+        @Override
+        public int nextDoc() throws IOException {
+          return advance(doc + 1);
+        }
 
-    void reset() {
-      index = -1;
-      doc = -1;
-    }
+        @Override
+        public int advance(int target) throws IOException {
+          if (target >= maxDoc) {
+            return doc = NO_MORE_DOCS;
+          }
+          return doc = target;
+        }
 
-    @Override
-    public int docID() {
-      return doc;
-    }
+        @Override
+        public long cost() {
+          return maxDoc;
+        }
 
-    @Override
-    public int nextDoc() throws IOException {
-      if (index >= docIDsLength - 1) {
-        index = docIDsLength;
-        return doc = NO_MORE_DOCS;
-      }
-      return doc = (int) docIds.get(++index);
-    }
+      };
+    } else {
+      // sparse
+      final LongValues values = getNumericValues(entry);
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numValues);
+      return new NumericDocValues() {
 
-    @Override
-    public int advance(int target) throws IOException {
-      long loIndex = index;
-      long step = 1;
-      long hiIndex;
-      int hiDoc;
-
-      // gallop forward by exponentially growing the interval
-      // in order to find an interval so that the target doc
-      // is in ]lo, hi]. Compared to a regular binary search,
-      // this optimizes the case that the caller performs many
-      // advance calls by small deltas
-      do {
-        hiIndex = index + step;
-        if (hiIndex >= docIDsLength) {
-          hiIndex = docIDsLength;
-          hiDoc = NO_MORE_DOCS;
-          break;
-        }
-        hiDoc = (int) docIds.get(hiIndex);
-        if (hiDoc >= target) {
-          break;
+        @Override
+        public int advance(int target) throws IOException {
+          return disi.advance(target);
         }
-        step <<= 1;
-      } while (true);
-
-      // now binary search
-      while (loIndex + 1 < hiIndex) {
-        final long midIndex = (loIndex + 1 + hiIndex) >>> 1;
-        final int midDoc = (int) docIds.get(midIndex);
-        if (midDoc >= target) {
-          hiIndex = midIndex;
-          hiDoc = midDoc;
-        } else {
-          loIndex = midIndex;
+
+        @Override
+        public int nextDoc() throws IOException {
+          return disi.nextDoc();
         }
-      }
 
-      index = (int) hiIndex;
-      return doc = hiDoc;
-    }
+        @Override
+        public int docID() {
+          return disi.docID();
+        }
 
-    @Override
-    public long longValue() {
-      assert index >= 0;
-      assert index < docIDsLength;
-      return values.get(index);
-    }
+        @Override
+        public long cost() {
+          return disi.cost();
+        }
 
-    @Override
-    public long cost() {
-      return docIDsLength;
+        @Override
+        public long longValue() throws IOException {
+          return values.get(disi.index());
+        }
+      };
     }
   }
 
-  static class SparseNumericDocValuesRandomAccessWrapper extends LongValues {
-
-    final SparseNumericDocValues values;
-    final long missingValue;
-
-    SparseNumericDocValuesRandomAccessWrapper(SparseNumericDocValues values, long missingValue) {
-      this.values = values;
-      this.missingValue = missingValue;
-    }
-
-    @Override
-    public long get(long longIndex) {
-      final int index = Math.toIntExact(longIndex);
-      int doc = values.docID();
-      if (doc >= index) {
-        values.reset();
+  private LongValues getNumericValues(NumericEntry entry) throws IOException {
+    if (entry.bitsPerValue == 0) {
+      return new LongValues() {
+        @Override
+        public long get(long index) {
+          return entry.minValue;
+        }
+      };
+    } else {
+      final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
+      LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+      if (entry.gcd != 1) {
+        values = applyGcd(values, entry.gcd);
       }
-      assert values.docID() < index;
-      try {
-        doc = values.advance(index);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
+      if (entry.minValue != 0) {
+        values = applyDelta(values, entry.minValue);
       }
-      if (doc == index) {
-        return values.longValue();
-      } else {
-        return missingValue;
+      if (entry.table != null) {
+        values = applyTable(values, entry.table);
       }
-    }
-
-  }
-
-  LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException {
-    BinaryEntry bytes = binaries.get(field.name);
-    switch(bytes.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        return getFixedBinary(field, bytes);
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        return getVariableBinary(field, bytes);
-      case BINARY_PREFIX_COMPRESSED:
-        return getCompressedBinary(field, bytes);
-      default:
-        throw new AssertionError();
+      return values;
     }
   }
 
-  @Override
-  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryEntry be = binaries.get(field.name);
-    DocIdSetIterator docsWithField = getDocsWithField(be.missingOffset, maxDoc);
-    LegacyBinaryDocValues values = getLegacyBinary(field);
-    return new BinaryDocValues() {
-
-      @Override
-      public int nextDoc() throws IOException {
-        return docsWithField.nextDoc();
-      }
-
+  private LongValues applyDelta(LongValues values, long delta) {
+    return new LongValues() {
       @Override
-      public int docID() {
-        return docsWithField.docID();
-      }
-
-      @Override
-      public long cost() {
-        return docsWithField.cost();
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        return docsWithField.advance(target);
-      }
-
-      @Override
-      public BytesRef binaryValue() {
-        return values.get(docsWithField.docID());
+      public long get(long index) {
+        return delta + values.get(index);
       }
     };
   }
 
-  private LegacyBinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
-
-    final BytesRef term = new BytesRef(bytes.maxLength);
-    final byte[] buffer = term.bytes;
-    final int length = term.length = bytes.maxLength;
-
-    return new LongBinaryDocValues() {
+  private LongValues applyGcd(LongValues values, long gcd) {
+    return new LongValues() {
       @Override
-      public BytesRef get(long id) {
-        try {
-          data.seek(id * length);
-          data.readBytes(buffer, 0, buffer.length);
-          return term;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
+      public long get(long index) {
+        return values.get(index) * gcd;
       }
     };
   }
 
-  private LegacyBinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final RandomAccessInput addressesData = this.data.randomAccessSlice(bytes.addressesOffset, bytes.addressesEndOffset - bytes.addressesOffset);
-    final LongValues addresses = DirectMonotonicReader.getInstance(bytes.addressesMeta, addressesData);
-
-    final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
-    final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
-    final byte buffer[] = term.bytes;
-
-    return new LongBinaryDocValues() {
+  private LongValues applyTable(LongValues values, long[] table) {
+    return new LongValues() {
       @Override
-      public BytesRef get(long id) {
-        long startAddress = addresses.get(id);
-        long endAddress = addresses.get(id+1);
-        int length = (int) (endAddress - startAddress);
-        try {
-          data.seek(startAddress);
-          data.readBytes(buffer, 0, length);
-          term.length = length;
-          return term;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
+      public long get(long index) {
+        return table[(int) values.get(index)];
       }
     };
   }
 
-  /** returns an address instance for prefix-compressed binary values. */
-  private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
-    if (addresses == null) {
-      data.seek(bytes.addressesOffset);
-      final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
-      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      if (!merging) {
-        addressInstances.put(field.name, addresses);
-        ramBytesUsed.addAndGet(addresses.ramBytesUsed() + Integer.BYTES);
-      }
-    }
-    return addresses;
-  }
-
-  /** returns a reverse lookup instance for prefix-compressed binary values. */
-  private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    ReverseTermsIndex index = reverseIndexInstances.get(field.name);
-    if (index == null) {
-      index = new ReverseTermsIndex();
-      data.seek(bytes.reverseIndexOffset);
-      long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
-      index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      long dataSize = data.readVLong();
-      PagedBytes pagedBytes = new PagedBytes(15);
-      pagedBytes.copy(data, dataSize);
-      index.terms = pagedBytes.freeze(true);
-      if (!merging) {
-        reverseIndexInstances.put(field.name, index);
-        ramBytesUsed.addAndGet(index.ramBytesUsed());
-      }
-    }
-    return index;
-  }
-
-  private LegacyBinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
-    final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
-    assert addresses.size() > 0; // we don't have to handle empty case
-    IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
-    return new CompressedBinaryDocValues(bytes, addresses, index, slice);
-  }
-
   @Override
-  public SortedDocValues getSorted(FieldInfo field) throws IOException {
-    final int valueCount = (int) binaries.get(field.name).count;
-    final LegacyBinaryDocValues binary = getLegacyBinary(field);
-    NumericEntry entry = ords.get(field.name);
-    final LongValues ordinals = getNumeric(entry);
-    if (entry.format == SPARSE_COMPRESSED) {
-      final SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) ordinals).values;
-      return new SortedDocValues() {
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryEntry entry = binaries.get(field.name);
+    if (entry.docsWithFieldOffset == -2) {
+      return DocValues.emptyBinary();
+    }
 
+    IndexInput bytesSlice = data.slice("fixed-binary", entry.dataOffset, entry.dataLength);
+    BytesRefs bytesRefs;
+    if (entry.minLength == entry.maxLength) {
+      bytesRefs = new BytesRefs() {
+        BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength);
         @Override
-        public int ordValue() {
-          return (int) sparseValues.longValue();
+        public BytesRef get(int index) throws IOException {
+          bytesSlice.seek((long) index * bytes.length);
+          bytesSlice.readBytes(bytes.bytes, 0, bytes.length);
+          return bytes;
         }
-
+      };
+    } else {
+      final RandomAccessInput addressesData = this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
+      final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
+      bytesRefs = new BytesRefs() {
+        BytesRef bytes = new BytesRef(entry.maxLength);
         @Override
-        public BytesRef lookupOrd(int ord) {
-          return binary.get(ord);
+        BytesRef get(int index) throws IOException {
+          long startOffset = addresses.get(index);
+          bytes.length = (int) (addresses.get(index + 1L) - startOffset);
+          bytesSlice.seek(startOffset);
+          bytesSlice.readBytes(bytes.bytes, 0, bytes.length);
+          return bytes;
         }
+      };
+    }
+
+    if (entry.docsWithFieldOffset == -1) {
+      // dense
+      return new BinaryDocValues() {
+
+        int doc = -1;
 
         @Override
-        public int getValueCount() {
-          return valueCount;
+        public int nextDoc() throws IOException {
+          return advance(doc + 1);
         }
 
         @Override
         public int docID() {
-          return sparseValues.docID();
+          return doc;
         }
 
         @Override
-        public int nextDoc() throws IOException {
-          return sparseValues.nextDoc();
+        public long cost() {
+          return maxDoc;
         }
 
         @Override
         public int advance(int target) throws IOException {
-          return sparseValues.advance(target);
+          if (target >= maxDoc) {
+            return doc = NO_MORE_DOCS;
+          }
+          return doc = target;
         }
 
         @Override
-        public long cost() {
-          return sparseValues.cost();
+        public BytesRef binaryValue() throws IOException {
+          return bytesRefs.get(doc);
         }
-
       };
-    }
-    return new SortedDocValues() {
-      private int docID = -1;
-      private int ord;
+    } else {
+      // sparse
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
+      return new BinaryDocValues() {
 
-      @Override
-      public int docID() {
-        return docID;
-      }
-
-      @Override
-      public int nextDoc() throws IOException {
-        assert docID != NO_MORE_DOCS;
-        while (true) {
-          docID++;
-          if (docID == maxDoc) {
-            docID = NO_MORE_DOCS;
-            break;
-          }
-          ord = (int) ordinals.get(docID);
-          if (ord != -1) {
-            break;
-          }
+        @Override
+        public int nextDoc() throws IOException {
+          return disi.nextDoc();
         }
-        return docID;
-      }
 
-      @Override
-      public int advance(int target) throws IOException {
-        if (target >= maxDoc) {
-          docID = NO_MORE_DOCS;
-          return docID;
-        } else {
-          docID = target-1;
-          return nextDoc();
+        @Override
+        public int docID() {
+          return disi.docID();
         }
-      }
-          
-      @Override
-      public int ordValue() {
-        return ord;
-      }
-
-      @Override
-      public long cost() {
-        // TODO
-        return 0;
-      }
-
-      @Override
-      public BytesRef lookupOrd(int ord) {
-        return binary.get(ord);
-      }
 
-      @Override
-      public int getValueCount() {
-        return valueCount;
-      }
+        @Override
+        public long cost() {
+          return disi.cost();
+        }
 
-      @Override
-      public int lookupTerm(BytesRef key) throws IOException {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
+        @Override
+        public int advance(int target) throws IOException {
+          return disi.advance(target);
         }
-      }
 
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
+        @Override
+        public BytesRef binaryValue() throws IOException {
+          return bytesRefs.get(disi.index());
         }
-      }
-    };
+      };
+    }
   }
 
-  /** returns an address instance for sortedset ordinal lists */
-  private LongValues getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
-    RandomAccessInput data = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-    return DirectMonotonicReader.getInstance(entry.monotonicMeta, data);
+  private static abstract class BytesRefs {
+    abstract BytesRef get(int index) throws IOException;
   }
 
   @Override
-  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedNumerics.get(field.name);
-    if (ss.format == SORTED_SINGLE_VALUED) {
-      NumericEntry numericEntry = numerics.get(field.name);
-      final LongValues values = getNumeric(numericEntry);
-      if (numericEntry.format == SPARSE_COMPRESSED) {
-        SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) values).values;
-        return new SortedNumericDocValues() {
-
-          @Override
-          public long nextValue() throws IOException {
-            return sparseValues.longValue();
-          }
-
-          @Override
-          public int docValueCount() {
-            return 1;
-          }
-
-          @Override
-          public int docID() {
-            return sparseValues.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return sparseValues.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return sparseValues.advance(target);
-          }
-
-          @Override
-          public long cost() {
-            return sparseValues.cost();
-          }
+  public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    SortedEntry entry = sorted.get(field.name);
+    return getSorted(entry);
+  }
 
-        };
-      }
-      final DocIdSetIterator docsWithField = getDocsWithField(numericEntry.missingOffset, maxDoc);
-      
-      return new SortedNumericDocValues() {
+  private SortedDocValues getSorted(SortedEntry entry) throws IOException {
+    if (entry.docsWithFieldOffset == -2) {
+      return DocValues.emptySorted();
+    }
 
+    final LongValues ords;
+    if (entry.bitsPerValue == 0) {
+      ords = new LongValues() {
         @Override
-        public int docID() {
-          return docsWithField.docID();
+        public long get(long index) {
+          return 0L;
         }
+      };
+    } else {
+      final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength);
+      ords = DirectReader.getInstance(slice, entry.bitsPerValue);
+    }
+
+    if (entry.docsWithFieldOffset == -1) {
+      // dense
+      return new BaseSortedDocValues(entry, data) {
+
+        int doc = -1;
 
         @Override
         public int nextDoc() throws IOException {
-          return docsWithField.nextDoc();
+          return advance(doc + 1);
         }
 
         @Override
-        public int advance(int target) throws IOException {
-          return docsWithField.advance(target);
+        public int docID() {
+          return doc;
         }
 
         @Override
         public long cost() {
-          return docsWithField.cost();
+          return maxDoc;
         }
 
         @Override
-        public int docValueCount() {
-          return 1;
+        public int advance(int target) throws IOException {
+          if (target >= maxDoc) {
+            return doc = NO_MORE_DOCS;
+          }
+          return doc = target;
         }
 
         @Override
-        public long nextValue() {
-          return values.get(docsWithField.docID());
+        public int ordValue() {
+          return (int) ords.get(doc);
         }
       };
-    } else if (ss.format == SORTED_WITH_ADDRESSES) {
-      NumericEntry numericEntry = numerics.get(field.name);
-      final LongValues values = getNumeric(numericEntry);
-      final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
-
-      return new SortedNumericDocValues() {
-        long startOffset;
-        long endOffset;
-        int docID = -1;
-        long upto;
+    } else {
+      // sparse
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
+      return new BaseSortedDocValues(entry, data) {
 
         @Override
-        public int docID() {
-          return docID;
+        public int nextDoc() throws IOException {
+          return disi.nextDoc();
         }
 
         @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              return docID;
-            }
-            startOffset = ordIndex.get(docID);
-            endOffset = ordIndex.get(docID+1L);
-            if (endOffset > startOffset) {
-              break;
-            }
-          }
-          upto = startOffset;
-          return docID;
+        public int docID() {
+          return disi.docID();
         }
 
         @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-        
-        @Override
         public long cost() {
-          // TODO
-          return 0;
+          return disi.cost();
         }
-        
+
         @Override
-        public int docValueCount() {
-          return (int) (endOffset - startOffset);
+        public int advance(int target) throws IOException {
+          return disi.advance(target);
         }
-        
+
         @Override
-        public long nextValue() {
-          return values.get(upto++);
+        public int ordValue() {
+          return (int) ords.get(disi.index());
         }
       };
-    } else if (ss.format == SORTED_SET_TABLE) {
-      NumericEntry entry = ords.get(field.name);
-      final LongValues ordinals = getNumeric(entry);
+    }
+  }
 
-      final long[] table = ss.table;
-      final int[] offsets = ss.tableOffsets;
-      return new SortedNumericDocValues() {
-        int startOffset;
-        int endOffset;
-        int docID = -1;
-        int upto;
+  private static abstract class BaseSortedDocValues extends SortedDocValues {
 
-        @Override
-        public int docID() {
-          return docID;
-        }
+    final SortedEntry entry;
+    final IndexInput data;
+    final TermsEnum termsEnum;
 
-        @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              return docID;
-            }
-            int ord = (int) ordinals.get(docID);
-            startOffset = offsets[ord];
-            endOffset = offsets[ord+1];
-            if (endOffset > startOffset) {
-              break;
-            }
-          }
-          upto = startOffset;
-          return docID;
-        }
+    BaseSortedDocValues(SortedEntry entry, IndexInput data) throws IOException {
+      this.entry = entry;
+      this.data = data;
+      this.termsEnum = termsEnum();
+    }
 
-        @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-        
-        @Override
-        public long cost() {
-          // TODO
-          return 0;
-        }
+    @Override
+    public int getValueCount() {
+      return Math.toIntExact(entry.termsDictSize);
+    }
 
-        @Override
-        public int docValueCount() {
-          return endOffset - startOffset;
-        }
-        
-        @Override
-        public long nextValue() {
-          return table[upto++];
-        }
-      };
-    } else {
-      throw new AssertionError();
+    @Override
+    public BytesRef lookupOrd(int ord) throws IOException {
+      termsEnum.seekExact(ord);
+      return termsEnum.term();
     }
-  }
 
-  @Override
-  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedSets.get(field.name);
-    switch (ss.format) {
-      case SORTED_SINGLE_VALUED:
-        return DocValues.singleton(getSorted(field));
-      case SORTED_WITH_ADDRESSES:
-        return getSortedSetWithAddresses(field);
-      case SORTED_SET_TABLE:
-        return getSortedSetTable(field, ss);
-      default:
-        throw new AssertionError();
+    @Override
+    public int lookupTerm(BytesRef key) throws IOException {
+      SeekStatus status = termsEnum.seekCeil(key);
+      switch (status) {
+        case FOUND:
+          return Math.toIntExact(termsEnum.ord());
+        default:
+          return Math.toIntExact(-1L - termsEnum.ord());
+      }
+    }
+
+    @Override
+    public TermsEnum termsEnum() throws IOException {
+      return new TermsDict(entry, data);
     }
   }
 
-  private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
-    final long valueCount = binaries.get(field.name).count;
-    // we keep the byte[]s and list of ords on disk, these could be large
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
-    final LongValues ordinals = getNumeric(ords.get(field.name));
-    // but the addresses to the ord stream are in RAM
-    final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+  private static abstract class BaseSortedSetDocValues extends SortedSetDocValues {
 
-    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
-      long startOffset;
-      long offset;
-      long endOffset;
+    final SortedSetEntry entry;
+    final IndexInput data;
+    final TermsEnum termsEnum;
 
-      @Override
-      public long nextOrd() {
-        if (offset == endOffset) {
-          return NO_MORE_ORDS;
-        } else {
-          long ord = ordinals.get(offset);
-          offset++;
-          return ord;
-        }
-      }
+    BaseSortedSetDocValues(SortedSetEntry entry, IndexInput data) throws IOException {
+      this.entry = entry;
+      this.data = data;
+      this.termsEnum = termsEnum();
+    }
 
-      @Override
-      public void setDocument(int docID) {
-        startOffset = offset = ordIndex.get(docID);
-        endOffset = ordIndex.get(docID+1L);
-      }
+    @Override
+    public long getValueCount() {
+      return entry.termsDictSize;
+    }
 
-      @Override
-      public BytesRef lookupOrd(long ord) {
-        return binary.get(ord);
-      }
+    @Override
+    public BytesRef lookupOrd(long ord) throws IOException {
+      termsEnum.seekExact(ord);
+      return termsEnum.term();
+    }
 
-      @Override
-      public long getValueCount() {
-        return valueCount;
+    @Override
+    public long lookupTerm(BytesRef key) throws IOException {
+      SeekStatus status = termsEnum.seekCeil(key);
+      switch (status) {
+        case FOUND:
+          return termsEnum.ord();
+        default:
+          return -1L - termsEnum.ord();
       }
+    }
 
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
+    @Override
+    public TermsEnum termsEnum() throws IOException {
+      return new TermsDict(entry, data);
+    }
+  }
+
+  private static class TermsDict extends TermsEnum {
+
+    final TermsDictEntry entry;
+    final LongValues blockAddresses;
+    final IndexInput bytes;
+    final long blockMask;
+    final LongValues indexAddresses;
+    final IndexInput indexBytes;
+    final BytesRef term;
+    long ord = -1;
+
+    TermsDict(TermsDictEntry entry, IndexInput data) throws IOException {
+      this.entry = entry;
+      RandomAccessInput addressesSlice = data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength);
+      blockAddresses = DirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice);
+      bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength);
+      blockMask = (1L << entry.termsDictBlockShift) - 1;
+      RandomAccessInput indexAddressesSlice = data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength);
+      indexAddresses = DirectMonotonicReader.getInstance(entry.termsIndexAddressesMeta, indexAddressesSlice);
+      indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength);
+      term = new BytesRef(entry.maxTermLength);
+    }
+
+    @Override
+    public BytesRef next() throws IOException {
+      if (++ord >= entry.termsDictSize) {
+        return null;
+      }
+      if ((ord & blockMask) == 0L) {
+        term.length = bytes.readVInt();
+        bytes.readBytes(term.bytes, 0, term.length);
+      } else {
+        final int token = Byte.toUnsignedInt(bytes.readByte());
+        int prefixLength = token & 0x0F;
+        int suffixLength = 1 + (token >>> 4);
+        if (prefixLength == 15) {
+          prefixLength += bytes.readVInt();
+        }
+        if (suffixLength == 16) {
+          suffixLength += bytes.readVInt();
         }
+        term.length = prefixLength + suffixLength;
+        bytes.readBytes(term.bytes, prefixLength, suffixLength);
       }
+      return term;
+    }
 
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+    @Override
+    public void seekExact(long ord) throws IOException {
+      if (ord < 0 || ord >= entry.termsDictSize) {
+        throw new IndexOutOfBoundsException();
+      }
+      final long blockIndex = ord >>> entry.termsDictBlockShift;
+      final long blockAddress = blockAddresses.get(blockIndex);
+      bytes.seek(blockAddress);
+      this.ord = (blockIndex << entry.termsDictBlockShift) - 1;
+      do {
+        next();
+      } while (this.ord < ord);
+    }
+
+    private BytesRef getTermFromIndex(long index) throws IOException {
+      assert index >= 0 && index <= (entry.termsDictSize - 1) >>> entry.termsDictIndexShift;
+      final long start = indexAddresses.get(index);
+      term.length = (int) (indexAddresses.get(index + 1) - start);
+      indexBytes.seek(start);
+      indexBytes.readBytes(term.bytes, 0, term.length);
+      return term;
+    }
+
+    private long seekTermsIndex(BytesRef text) throws IOException {
+      long lo = 0L;
+      long hi = (entry.termsDictSize - 1) >>> entry.termsDictIndexShift;
+      while (lo <= hi) {
+        final long mid = (lo + hi) >>> 1;
+        getTermFromIndex(mid);
+        final int cmp = term.compareTo(text);
+        if (cmp <= 0) {
+          lo = mid + 1;
         } else {
-          return super.termsEnum();
+          hi = mid - 1;
         }
       }
-      }, maxDoc);
-  }
 
-  private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
-    final long valueCount = binaries.get(field.name).count;
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
-    final NumericEntry ordinalsEntry = ords.get(field.name);
-    final LongValues ordinals = getNumeric(ordinalsEntry);
+      assert hi < 0 || getTermFromIndex(hi).compareTo(text) <= 0;
+      assert hi == ((entry.termsDictSize - 1) >>> entry.termsDictIndexShift) || getTermFromIndex(hi + 1).compareTo(text) > 0;
 
-    final long[] table = ss.table;
-    final int[] offsets = ss.tableOffsets;
-
-    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
+      return hi;
+    }
 
-      int offset, startOffset, endOffset;
+    private BytesRef getFirstTermFromBlock(long block) throws IOException {
+      assert block >= 0 && block <= (entry.termsDictSize - 1) >>> entry.termsDictBlockShift;
+      final long blockAddress = blockAddresses.get(block);
+      bytes.seek(blockAddress);
+      term.length = bytes.readVInt();
+      bytes.readBytes(term.bytes, 0, term.length);
+      return term;
+    }
 
-      @Override
-      public void setDocument(int docID) {
-        final int ord = (int) ordinals.get(docID);
-        offset = startOffset = offsets[ord];
-        endOffset = offsets[ord + 1];
+    private long seekBlock(BytesRef text) throws IOException {
+      long index = seekTermsIndex(text);
+      if (index == -1L) {
+        return -1L;
       }
 
-      @Override
-      public long nextOrd() {
-        if (offset == endOffset) {
-          return NO_MORE_ORDS;
+      long ordLo = index << entry.termsDictIndexShift;
+      long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L;
+
+      long blockLo = ordLo >>> entry.termsDictBlockShift;
+      long blockHi = ordHi >>> entry.termsDictBlockShift;
+
+      while (blockLo <= blockHi) {
+        final long blockMid = (blockLo + blockHi) >>> 1;
+        getFirstTermFromBlock(blockMid);
+        final int cmp = term.compareTo(text);
+        if (cmp <= 0) {
+          blockLo = blockMid + 1;
         } else {
-          return table[offset++];
+          blockHi = blockMid - 1;
         }
       }
 
-      @Override
-      public BytesRef lookupOrd(long ord) {
-        return binary.get(ord);
-      }
+      assert blockHi < 0 || getFirstTermFromBlock(blockHi).compareTo(text) <= 0;
+      assert blockHi == ((entry.termsDictSize - 1) >>> entry.termsDictBlockShift) || getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0;
 
-      @Override
-      public long getValueCount() {
-        return valueCount;
-      }
+      return blockHi;
+    }
 
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues) binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
+    @Override
+    public SeekStatus seekCeil(BytesRef text) throws IOException {
+      final long block = seekBlock(text);
+      if (block == -1) {
+        // before the first term
+        seekExact(0L);
+        return SeekStatus.NOT_FOUND;
+      }
+      final long blockAddress = blockAddresses.get(block);
+      this.ord = block << entry.termsDictBlockShift;
+      bytes.seek(blockAddress);
+      term.length = bytes.readVInt();
+      bytes.readBytes(term.bytes, 0, term.length);
+      while (true) {
+        int cmp = term.compareTo(text);
+        if (cmp == 0) {
+          return SeekStatus.FOUND;
+        } else if (cmp > 0) {
+          return SeekStatus.NOT_FOUND;
+        }
+        if (next() == null) {
+          return SeekStatus.END;
         }
       }
+    }
 
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues) binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-      }, maxDoc);
-  }
+    @Override
+    public BytesRef term() throws IOException {
+      return term;
+    }
 
-  private Bits getLiveBits(final long offset, final int count) throws IOException {
-    if (offset == ALL_MISSING) {
-      return new Bits.MatchNoBits(count);
-    } else if (offset == ALL_LIVE) {
-      return new Bits.MatchAllBits(count);
-    } else {
-      int length = (int) ((count + 63L) >>> 6);
-      final RandomAccessInput in = data.randomAccessSlice(offset, length << 3);
-      return new Bits() {
+    @Override
+    public long ord() throws IOException {
+      return ord;
+    }
 
-        int wordIndex = -1;
-        long word = 0;
+    @Override
+    public long totalTermFreq() throws IOException {
+      return -1L;
+    }
 
-        @Override
-        public boolean get(int index) {
-          try {
-            int i = index >>> 6;
-            if (wordIndex != i) {
-              wordIndex = i;
-              word = in.readLong(i << 3);
-            }
-            return (word & (1L << index)) != 0;
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
+    @Override
+    public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+      throw new UnsupportedOperationException();
+    }
 
-        @Override
-        public int length() {
-          return count;
-        }
-      };
+    @Override
+    public int docFreq() throws IOException {
+      throw new UnsupportedOperationException();
     }
   }
 
-  private DocIdSetIterator getDocsWithField(final long offset, final int count) throws IOException {
-    if (offset == ALL_MISSING) {
-      return DocIdSetIterator.empty();
-    } else if (offset == ALL_LIVE) {
-      return DocIdSetIterator.all(count);
-    } else {
-      int length = (int) ((count + 63L) >>> 6);
-      final RandomAccessInput in = data.randomAccessSlice(offset, length << 3);
-      return new DocIdSetIterator() {
+  @Override
+  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+    SortedNumericEntry entry = sortedNumerics.get(field.name);
+    if (entry.numValues == entry.numDocsWithField) {
+      return DocValues.singleton(getNumeric(entry));
+    }
 
-        int doc = -1;
-        int wordIndex = -1;
-        long word = 0;
+    final RandomAccessInput addressesInput = data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
+    final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput);
 
-        private int nextSetBit(int index) throws IOException {
-          int i = index >>> 6;
+    final LongValues values = getNumericValues(entry);
 
-          if (wordIndex != i) {
-            wordIndex = i;
-            word = in.readLong(i << 3);
-          }
-          long w = word >>> index;
-          if (w != 0) {
-            return index + Long.numberOfTrailingZeros(w);
-          }
-
-          while (++i < length) {
-            wordIndex = i;
-            word = in.readLong(i << 3);
-            if (word != 0) {
-              return (i << 6) + Long.numberOfTrailingZeros(word);
-            }
-          }
+    if (entry.docsWithFieldOffset == -1) {
+      // dense
+      return new SortedNumericDocValues() {
 
-          return DocIdSetIterator.NO_MORE_DOCS;
-        }
+        int doc = -1;
+        long start, end;
+        int count;
 
         @Override
         public int nextDoc() throws IOException {
-          return advance(docID() + 1);
+          return advance(doc + 1);
         }
 
         @Override
@@ -1422,395 +954,196 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
 
         @Override
         public long cost() {
-          // TODO: what is the exact cardinality
-          return count;
+          return maxDoc;
         }
 
         @Override
         public int advance(int target) throws IOException {
-          if (target >= count) {
+          if (target >= maxDoc) {
             return doc = NO_MORE_DOCS;
           }
-          return doc = nextSetBit(target);
+          start = addresses.get(target);
+          end = addresses.get(target + 1L);
+          count = (int) (end - start);
+          return doc = target;
         }
-      };
-    }
-  }
-
-  private SparseNumericDocValues getSparseNumericDocValues(NumericEntry entry) throws IOException {
-    final RandomAccessInput docIdsData = this.data.randomAccessSlice(entry.missingOffset, entry.offset - entry.missingOffset);
-    final LongValues docIDs = DirectMonotonicReader.getInstance(entry.monotonicMeta, docIdsData);
-    final LongValues values = getNumeric(entry.nonMissingValues); // cannot be sparse
-    return new SparseNumericDocValues(Math.toIntExact(entry.numDocsWithValue), docIDs, values);
-  }
-
-  @Override
-  public synchronized DocValuesProducer getMergeInstance() throws IOException {
-    return new Lucene70DocValuesProducer(this);
-  }
-
-  @Override
-  public void close() throws IOException {
-    data.close();
-  }
-
-  /** metadata entry for a numeric docvalues field */
-  static class NumericEntry {
-    private NumericEntry() {}
-    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
-    long missingOffset;
-    /** offset to the actual numeric values */
-    public long offset;
-    /** end offset to the actual numeric values */
-    public long endOffset;
-    /** bits per value used to pack the numeric values */
-    public int bitsPerValue;
-
-    int format;
-    /** count of values written */
-    public long count;
-
-    /** monotonic meta */
-    public DirectMonotonicReader.Meta monotonicMeta;
-
-    long minValue;
-    long gcd;
-    long table[];
 
-    /** for sparse compression */
-    long numDocsWithValue;
-    NumericEntry nonMissingValues;
-    NumberType numberType;
+        @Override
+        public long nextValue() throws IOException {
+          return values.get(start++);
+        }
 
-  }
+        @Override
+        public int docValueCount() {
+          return count;
+        }
+      };
+    } else {
+      // sparse
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
+      return new SortedNumericDocValues() {
 
-  /** metadata entry for a binary docvalues field */
-  static class BinaryEntry {
-    private BinaryEntry() {}
-    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
-    long missingOffset;
-    /** offset to the actual binary values */
-    long offset;
-
-    int format;
-    /** count of values written */
-    public long count;
-    int minLength;
-    int maxLength;
-    /** offset to the addressing data that maps a value to its slice of the byte[] */
-    public long addressesOffset, addressesEndOffset;
-    /** meta data for addresses */
-    public DirectMonotonicReader.Meta addressesMeta;
-    /** offset to the reverse index */
-    public long reverseIndexOffset;
-    /** packed ints version used to encode addressing information */
-    public int packedIntsVersion;
-    /** packed ints blocksize */
-    public int blockSize;
-  }
+        boolean set;
+        long start, end;
+        int count;
 
-  /** metadata entry for a sorted-set docvalues field */
-  static class SortedSetEntry {
-    private SortedSetEntry() {}
-    int format;
+        @Override
+        public int nextDoc() throws IOException {
+          set = false;
+          return disi.nextDoc();
+        }
 
-    long[] table;
-    int[] tableOffsets;
-  }
+        @Override
+        public int docID() {
+          return disi.docID();
+        }
 
-  // internally we compose complex dv (sorted/sortedset) from other ones
-  static abstract class LongBinaryDocValues extends LegacyBinaryDocValues {
-    @Override
-    public final BytesRef get(int docID) {
-      return get((long)docID);
-    }
+        @Override
+        public long cost() {
+          return disi.cost();
+        }
 
-    abstract BytesRef get(long id);
-  }
+        @Override
+        public int advance(int target) throws IOException {
+          set = false;
+          return disi.advance(target);
+        }
 
-  // used for reverse lookup to a small range of blocks
-  static class ReverseTermsIndex implements Accountable {
-    public MonotonicBlockPackedReader termAddresses;
-    public PagedBytes.Reader terms;
+        @Override
+        public long nextValue() throws IOException {
+          set();
+          return values.get(start++);
+        }
 
-    @Override
-    public long ramBytesUsed() {
-      return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
-    }
+        @Override
+        public int docValueCount() {
+          set();
+          return count;
+        }
 
-    @Override
-    public Collection<Accountable> getChildResources() {
-      List<Accountable> resources = new ArrayList<>();
-      resources.add(Accountables.namedAccountable("term bytes", terms));
-      resources.add(Accountables.namedAccountable("term addresses", termAddresses));
-      return Collections.unmodifiableList(resources);
-    }
+        private void set() {
+          if (set == false) {
+            final int index = disi.index();
+            start = addresses.get(index);
+            end = addresses.get(index + 1L);
+            count = (int) (end - start);
+            set = true;
+          }
+        }
 
-    @Override
-    public String toString() {
-      return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
+      };
     }
   }
 
-  //in the compressed case, we add a few additional operations for
-  //more efficient reverse lookup and enumeration
-  static final class CompressedBinaryDocValues extends LongBinaryDocValues {
-    final long numValues;
-    final long numIndexValues;
-    final int maxTermLength;
-    final MonotonicBlockPackedReader addresses;
-    final IndexInput data;
-    final CompressedBinaryTermsEnum termsEnum;
-    final PagedBytes.Reader reverseTerms;
-    final MonotonicBlockPackedReader reverseAddresses;
-    final long numReverseIndexValues;
-
-    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
-      this.maxTermLength = bytes.maxLength;
-      this.numValues = bytes.count;
-      this.addresses = addresses;
-      this.numIndexValues = addresses.size();
-      this.data = data;
-      this.reverseTerms = index.terms;
-      this.reverseAddresses = index.termAddresses;
-      this.numReverseIndexValues = reverseAddresses.size();
-      this.termsEnum = getTermsEnum(data);
-    }
-
-    @Override
-    public BytesRef get(long id) {
-      try {
-        termsEnum.seekExact(id);
-        return termsEnum.term();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    SortedSetEntry entry = sortedSets.get(field.name);
+    if (entry.singleValueEntry != null) {
+      return DocValues.singleton(getSorted(entry.singleValueEntry));
     }
 
-    long lookupTerm(BytesRef key) {
-      try {
-        switch (termsEnum.seekCeil(key)) {
-          case FOUND: return termsEnum.ord();
-          case NOT_FOUND: return -termsEnum.ord()-1;
-          default: return -numValues-1;
-        }
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
-      }
-    }
+    final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength);
+    final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue);
 
-    TermsEnum getTermsEnum() {
-      try {
-        return getTermsEnum(data.clone());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
+    final RandomAccessInput addressesInput = data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
+    final LongValues addresses = DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput);
 
-    private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
-      return new CompressedBinaryTermsEnum(input);
-    }
+    if (entry.docsWithFieldOffset == -1) {
+      // dense
+      return new BaseSortedSetDocValues(entry, data) {
 
-    class CompressedBinaryTermsEnum extends TermsEnum {
-      private long currentOrd = -1;
-      // offset to the start of the current block
-      private long currentBlockStart;
-      private final IndexInput input;
-      // delta from currentBlockStart to start of each term
-      private final int offsets[] = new int[INTERVAL_COUNT];
-      private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
-
-      private final BytesRef term = new BytesRef(maxTermLength);
-      private final BytesRef firstTerm = new BytesRef(maxTermLength);
-      private final BytesRef scratch = new BytesRef();
-
-      CompressedBinaryTermsEnum(IndexInput input) throws IOException {
-        this.input = input;
-        input.seek(0);
-      }
+        int doc = -1;
+        long start;
+        long end;
 
-      private void readHeader() throws IOException {
-        firstTerm.length = input.readVInt();
-        input.readBytes(firstTerm.bytes, 0, firstTerm.length);
-        input.readBytes(buffer, 0, INTERVAL_COUNT-1);
-        if (buffer[0] == -1) {
-          readShortAddresses();
-        } else {
-          readByteAddresses();
+        @Override
+        public int nextDoc() throws IOException {
+          return advance(doc + 1);
         }
-        currentBlockStart = input.getFilePointer();
-      }
 
-      // read single byte addresses: each is delta - 2
-      // (shared prefix byte and length > 0 are both implicit)
-      private void readByteAddresses() throws IOException {
-        int addr = 0;
-        for (int i = 1; i < offsets.length; i++) {
-          addr += 2 + (buffer[i-1] & 0xFF);
-          offsets[i] = addr;
-        }
-      }
-
-      // read double byte addresses: each is delta - 2
-      // (shared prefix byte and length > 0 are both implicit)
-      private void readShortAddresses() throws IOException {
-        input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
-        int addr = 0;
-        for (int i = 1; i < offsets.length; i++) {
-          int x = i<<1;
-          addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
-          offsets[i] = addr;
+        @Override
+        public int docID() {
+          return doc;
         }
-      }
-
-      // set term to the first term
-      private void readFirstTerm() throws IOException {
-        term.length = firstTerm.length;
-        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
-      }
-
-      // read term at offset, delta encoded from first term
-      private void readTerm(int offset) throws IOException {
-        int start = input.readByte() & 0xFF;
-        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
-        int suffix = offsets[offset] - offsets[offset-1] - 1;
-        input.readBytes(term.bytes, start, suffix);
-        term.length = start + suffix;
-      }
 
-      @Override
-      public BytesRef next() throws IOException {
-        currentOrd++;
-        if (currentOrd >= numValues) {
-          return null;
-        } else {
-          int offset = (int) (currentOrd & INTERVAL_MASK);
-          if (offset == 0) {
-            // switch to next block
-            readHeader();
-            readFirstTerm();
-          } else {
-            readTerm(offset);
-          }
-          return term;
+        @Override
+        public long cost() {
+          return maxDoc;
         }
-      }
 
-      // binary search reverse index to find smaller
-      // range of blocks to search
-      long binarySearchIndex(BytesRef text) throws IOException {
-        long low = 0;
-        long high = numReverseIndexValues - 1;
-        while (low <= high) {
-          long mid = (low + high) >>> 1;
-          reverseTerms.fill(scratch, reverseAddresses.get(mid));
-          int cmp = scratch.compareTo(text);
-
-          if (cmp < 0) {
-            low = mid + 1;
-          } else if (cmp > 0) {
-            high = mid - 1;
-          } else {
-            return mid;
+        @Override
+        public int advance(int target) throws IOException {
+          if (target >= maxDoc) {
+            return doc = NO_MORE_DOCS;
           }
+          start = addresses.get(target);
+          end = addresses.get(target + 1L);
+          return doc = target;
         }
-        return high;
-      }
 
-      // binary search against first term in block range
-      // to find term's block
-      long binarySearchBlock(BytesRef text, long low, long high) throws IOException {
-        while (low <= high) {
-          long mid = (low + high) >>> 1;
-          input.seek(addresses.get(mid));
-          term.length = input.readVInt();
-          input.readBytes(term.bytes, 0, term.length);
-          int cmp = term.compareTo(text);
-
-          if (cmp < 0) {
-            low = mid + 1;
-          } else if (cmp > 0) {
-            high = mid - 1;
-          } else {
-            return mid;
+        @Override
+        public long nextOrd() throws IOException {
+          if (start == end) {
+            return NO_MORE_ORDS;
           }
+          return ords.get(start++);
         }
-        return high;
-      }
 
-      @Override
-      public SeekStatus seekCeil(BytesRef text) throws IOException {
-        // locate block: narrow to block range with index, then search blocks
-        final long block;
-        long indexPos = binarySearchIndex(text);
-        if (indexPos < 0) {
-          block = 0;
-        } else {
-          long low = indexPos << BLOCK_INTERVAL_SHIFT;
-          long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
-          block = Math.max(low, binarySearchBlock(text, low, high));
-        }
+      };
+    } else {
+      // sparse
+      final IndexedDISI disi = new IndexedDISI(data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
+      return new BaseSortedSetDocValues(entry, data) {
 
-        // position before block, then scan to term.
-        input.seek(addresses.get(block));
-        currentOrd = (block << INTERVAL_SHIFT) - 1;
+        boolean set;
+        long start;
+        long end = 0;
 
-        while (next() != null) {
-          int cmp = term.compareTo(text);
-          if (cmp == 0) {
-            return SeekStatus.FOUND;
-          } else if (cmp > 0) {
-            return SeekStatus.NOT_FOUND;
-          }
+        @Override
+        public int nextDoc() throws IOException {
+          set = false;
+          return disi.nextDoc();
         }
-        return SeekStatus.END;
-      }
 
-      @Override
-      public void seekExact(long ord) throws IOException {
-        long block = ord >>> INTERVAL_SHIFT;
-        if (block != currentOrd >>> INTERVAL_SHIFT) {
-          // switch to different block
-          input.seek(addresses.get(block));
-          readHeader();
+        @Override
+        public int docID() {
+          return disi.docID();
         }
 
-        currentOrd = ord;
-
-        int offset = (int) (ord & INTERVAL_MASK);
-        if (offset == 0) {
-          readFirstTerm();
-        } else {
-          input.seek(currentBlockStart + offsets[offset-1]);
-          readTerm(offset);
+        @Override
+        public long cost() {
+          return disi.cost();
         }
-      }
-
-      @Override
-      public BytesRef term() throws IOException {
-        return term;
-      }
-
-      @Over

<TRUNCATED>