You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2016/10/04 08:52:01 UTC

[2/3] lucene-solr:master: LUCENE-7453: Create a Lucene70Codec.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/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
new file mode 100644
index 0000000..4f80266
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
@@ -0,0 +1,1816 @@
+/*
+ * 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.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.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,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 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()));
+
+    int version = -1;
+    int numFields = -1;
+
+    // read in the entries from the metadata file.
+    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
+      Throwable priorE = null;
+      try {
+        version = CodecUtil.checkIndexHeader(in, metaCodec,
+                                        Lucene70DocValuesFormat.VERSION_START,
+                                        Lucene70DocValuesFormat.VERSION_CURRENT,
+                                        state.segmentInfo.getId(),
+                                        state.segmentSuffix);
+        numFields = readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
+    }
+
+    this.numFields = numFields;
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+    this.data = state.directory.openInput(dataName, state.context);
+    boolean success = false;
+    try {
+      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec,
+                                                 Lucene70DocValuesFormat.VERSION_START,
+                                                 Lucene70DocValuesFormat.VERSION_CURRENT,
+                                                 state.segmentInfo.getId(),
+                                                 state.segmentSuffix);
+      if (version != version2) {
+        throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data);
+      }
+
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(data);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.data);
+      }
+    }
+  }
+
+  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++;
+      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));
+      } else if (type == Lucene70DocValuesFormat.BINARY) {
+        BinaryEntry b = readBinaryEntry(info, meta);
+        binaries.put(info.name, b);
+      } else if (type == Lucene70DocValuesFormat.SORTED) {
+        readSortedField(info, 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();
+        }
+      } 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();
+        }
+      } else {
+        throw new CorruptIndexException("invalid type: " + type, meta);
+      }
+      fieldNumber = meta.readVInt();
+    }
+    return numFields;
+  }
+
+  private NumericEntry readNumericEntry(FieldInfo info, IndexInput 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);
+    }
+    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);
+        }
+
+        // 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);
+    }
+    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();
+        break;
+      default:
+        throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+    }
+    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);
+    }
+    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();
+          }
+
+          @Override
+          public int nextDoc() throws IOException {
+            return docsWithField.nextDoc();
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            return docsWithField.advance(target);
+          }
+
+          @Override
+          public long cost() {
+            return docsWithField.cost();
+          }
+
+        };
+      }
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed.get();
+  }
+
+  @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);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(data);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + numFields + ")";
+  }
+
+  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();
+    }
+  }
+
+  static final class SparseNumericDocValues extends NumericDocValues {
+
+    final int docIDsLength;
+    final LongValues docIds, values;
+
+    int index, doc;
+
+    SparseNumericDocValues(int docIDsLength, LongValues docIDs, LongValues values) {
+      this.docIDsLength = docIDsLength;
+      this.docIds = docIDs;
+      this.values = values;
+      reset();
+    }
+
+    void reset() {
+      index = -1;
+      doc = -1;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (index >= docIDsLength - 1) {
+        index = docIDsLength;
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = (int) docIds.get(++index);
+    }
+
+    @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;
+        }
+        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;
+        }
+      }
+
+      index = (int) hiIndex;
+      return doc = hiDoc;
+    }
+
+    @Override
+    public long longValue() {
+      assert index >= 0;
+      assert index < docIDsLength;
+      return values.get(index);
+    }
+
+    @Override
+    public long cost() {
+      return docIDsLength;
+    }
+  }
+
+  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();
+      }
+      assert values.docID() < index;
+      try {
+        doc = values.advance(index);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      if (doc == index) {
+        return values.longValue();
+      } else {
+        return missingValue;
+      }
+    }
+
+  }
+
+  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();
+    }
+  }
+
+  @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();
+      }
+
+      @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());
+      }
+    };
+  }
+
+  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() {
+      @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);
+        }
+      }
+    };
+  }
+
+  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() {
+      @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);
+        }
+      }
+    };
+  }
+
+  /** 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() {
+
+        @Override
+        public int ordValue() {
+          return (int) sparseValues.longValue();
+        }
+
+        @Override
+        public BytesRef lookupOrd(int ord) {
+          return binary.get(ord);
+        }
+
+        @Override
+        public int getValueCount() {
+          return valueCount;
+        }
+
+        @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();
+        }
+
+      };
+    }
+    return new SortedDocValues() {
+      private int docID = -1;
+      private int ord;
+
+      @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;
+          }
+        }
+        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 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 int lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+    };
+  }
+
+  /** 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);
+  }
+
+  @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();
+          }
+
+        };
+      }
+      final DocIdSetIterator docsWithField = getDocsWithField(numericEntry.missingOffset, maxDoc);
+      
+      return new SortedNumericDocValues() {
+
+        @Override
+        public int docID() {
+          return docsWithField.docID();
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          return docsWithField.nextDoc();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          return docsWithField.advance(target);
+        }
+
+        @Override
+        public long cost() {
+          return docsWithField.cost();
+        }
+
+        @Override
+        public int docValueCount() {
+          return 1;
+        }
+
+        @Override
+        public long nextValue() {
+          return values.get(docsWithField.docID());
+        }
+      };
+    } 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;
+
+        @Override
+        public int docID() {
+          return docID;
+        }
+
+        @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;
+        }
+
+        @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 docValueCount() {
+          return (int) (endOffset - startOffset);
+        }
+        
+        @Override
+        public long nextValue() {
+          return values.get(upto++);
+        }
+      };
+    } 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;
+
+        @Override
+        public int docID() {
+          return docID;
+        }
+
+        @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;
+        }
+
+        @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 docValueCount() {
+          return endOffset - startOffset;
+        }
+        
+        @Override
+        public long nextValue() {
+          return table[upto++];
+        }
+      };
+    } else {
+      throw new AssertionError();
+    }
+  }
+
+  @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();
+    }
+  }
+
+  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));
+
+    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
+      long startOffset;
+      long offset;
+      long endOffset;
+
+      @Override
+      public long nextOrd() {
+        if (offset == endOffset) {
+          return NO_MORE_ORDS;
+        } else {
+          long ord = ordinals.get(offset);
+          offset++;
+          return ord;
+        }
+      }
+
+      @Override
+      public void setDocument(int docID) {
+        startOffset = offset = ordIndex.get(docID);
+        endOffset = ordIndex.get(docID+1L);
+      }
+
+      @Override
+      public BytesRef lookupOrd(long ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public long getValueCount() {
+        return valueCount;
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+      }, 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);
+
+    final long[] table = ss.table;
+    final int[] offsets = ss.tableOffsets;
+
+    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
+
+      int offset, startOffset, endOffset;
+
+      @Override
+      public void setDocument(int docID) {
+        final int ord = (int) ordinals.get(docID);
+        offset = startOffset = offsets[ord];
+        endOffset = offsets[ord + 1];
+      }
+
+      @Override
+      public long nextOrd() {
+        if (offset == endOffset) {
+          return NO_MORE_ORDS;
+        } else {
+          return table[offset++];
+        }
+      }
+
+      @Override
+      public BytesRef lookupOrd(long ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public long getValueCount() {
+        return valueCount;
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues) binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues) binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+      }, maxDoc);
+  }
+
+  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() {
+
+        int wordIndex = -1;
+        long word = 0;
+
+        @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 int length() {
+          return count;
+        }
+      };
+    }
+  }
+
+  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() {
+
+        int doc = -1;
+        int wordIndex = -1;
+        long word = 0;
+
+        private int nextSetBit(int index) throws IOException {
+          int i = index >>> 6;
+
+          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);
+            }
+          }
+
+          return DocIdSetIterator.NO_MORE_DOCS;
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          return advance(docID() + 1);
+        }
+
+        @Override
+        public int docID() {
+          return doc;
+        }
+
+        @Override
+        public long cost() {
+          // TODO: what is the exact cardinality
+          return count;
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          if (target >= count) {
+            return doc = NO_MORE_DOCS;
+          }
+          return doc = nextSetBit(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;
+
+  }
+
+  /** 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;
+  }
+
+  /** metadata entry for a sorted-set docvalues field */
+  static class SortedSetEntry {
+    private SortedSetEntry() {}
+    int format;
+
+    long[] table;
+    int[] tableOffsets;
+  }
+
+  // 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);
+    }
+
+    abstract BytesRef get(long id);
+  }
+
+  // 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 ramBytesUsed() {
+      return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
+    }
+
+    @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);
+    }
+
+    @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);
+      }
+    }
+
+    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);
+      }
+    }
+
+    TermsEnum getTermsEnum() {
+      try {
+        return getTermsEnum(data.clone());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
+      return new CompressedBinaryTermsEnum(input);
+    }
+
+    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);
+      }
+
+      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();
+        }
+        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;
+        }
+      }
+
+      // 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;
+        }
+      }
+
+      // 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;
+          }
+        }
+        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;
+          }
+        }
+        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));
+        }
+
+        // position before block, then scan to term.
+        input.seek(addresses.get(block));
+        currentOrd = (block << INTERVAL_SHIFT) - 1;
+
+        while (next() != null) {
+          int cmp = term.compareTo(text);
+          if (cmp == 0) {
+            return SeekStatus.FOUND;
+          } else if (cmp > 0) {
+            return SeekStatus.NOT_FOUND;
+          }
+        }
+        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();
+        }
+
+        currentOrd = ord;
+
+        int offset = (int) (ord & INTERVAL_MASK);
+        if (offset == 0) {
+          readFirstTerm();
+        } else {
+          input.seek(currentBlockStart + offsets[offset-1]);
+          readTerm(offset);
+        }
+      }
+
+      @Override
+      public BytesRef term() throws IOException {
+        return term;
+      }
+
+      @Override
+      public long ord() throws IOException {
+        return currentOrd;
+      }
+
+      @Override
+      public int docFreq() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long totalTermFreq() throws IOException {
+        return -1;
+      }
+
+      @Override
+      public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
new file mode 100644
index 0000000..77492ad
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
@@ -0,0 +1,392 @@
+/*
+ * 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.
+ */
+
+/**
+ * Lucene 7.0 file format.
+ * 
+ * <h1>Apache Lucene - Index File Formats</h1>
+ * <div>
+ * <ul>
+ * <li><a href="#Introduction">Introduction</a></li>
+ * <li><a href="#Definitions">Definitions</a>
+ *   <ul>
+ *   <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
+ *   <li><a href="#Types_of_Fields">Types of Fields</a></li>
+ *   <li><a href="#Segments">Segments</a></li>
+ *   <li><a href="#Document_Numbers">Document Numbers</a></li>
+ *   </ul>
+ * </li>
+ * <li><a href="#Overview">Index Structure Overview</a></li>
+ * <li><a href="#File_Naming">File Naming</a></li>
+ * <li><a href="#file-names">Summary of File Extensions</a>
+ *   <ul>
+ *   <li><a href="#Lock_File">Lock File</a></li>
+ *   <li><a href="#History">History</a></li>
+ *   <li><a href="#Limitations">Limitations</a></li>
+ *   </ul>
+ * </li>
+ * </ul>
+ * </div>
+ * <a name="Introduction"></a>
+ * <h2>Introduction</h2>
+ * <div>
+ * <p>This document defines the index file formats used in this version of Lucene.
+ * If you are using a different version of Lucene, please consult the copy of
+ * <code>docs/</code> that was distributed with
+ * the version you are using.</p>
+ * <p>This document attempts to provide a high-level definition of the Apache
+ * Lucene file formats.</p>
+ * </div>
+ * <a name="Definitions"></a>
+ * <h2>Definitions</h2>
+ * <div>
+ * <p>The fundamental concepts in Lucene are index, document, field and term.</p>
+ * <p>An index contains a sequence of documents.</p>
+ * <ul>
+ * <li>A document is a sequence of fields.</li>
+ * <li>A field is a named sequence of terms.</li>
+ * <li>A term is a sequence of bytes.</li>
+ * </ul>
+ * <p>The same sequence of bytes in two different fields is considered a different 
+ * term. Thus terms are represented as a pair: the string naming the field, and the
+ * bytes within the field.</p>
+ * <a name="Inverted_Indexing"></a>
+ * <h3>Inverted Indexing</h3>
+ * <p>The index stores statistics about terms in order to make term-based search
+ * more efficient. Lucene's index falls into the family of indexes known as an
+ * <i>inverted index.</i> This is because it can list, for a term, the documents
+ * that contain it. This is the inverse of the natural relationship, in which
+ * documents list terms.</p>
+ * <a name="Types_of_Fields"></a>
+ * <h3>Types of Fields</h3>
+ * <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
+ * in the index literally, in a non-inverted manner. Fields that are inverted are
+ * called <i>indexed</i>. A field may be both stored and indexed.</p>
+ * <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
+ * text of a field may be used literally as a term to be indexed. Most fields are
+ * tokenized, but sometimes it is useful for certain identifier fields to be
+ * indexed literally.</p>
+ * <p>See the {@link org.apache.lucene.document.Field Field}
+ * java docs for more information on Fields.</p>
+ * <a name="Segments"></a>
+ * <h3>Segments</h3>
+ * <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
+ * Each segment is a fully independent index, which could be searched separately.
+ * Indexes evolve by:</p>
+ * <ol>
+ * <li>Creating new segments for newly added documents.</li>
+ * <li>Merging existing segments.</li>
+ * </ol>
+ * <p>Searches may involve multiple segments and/or multiple indexes, each index
+ * potentially composed of a set of segments.</p>
+ * <a name="Document_Numbers"></a>
+ * <h3>Document Numbers</h3>
+ * <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
+ * The first document added to an index is numbered zero, and each subsequent
+ * document added gets a number one greater than the previous.</p>
+ * <p>Note that a document's number may change, so caution should be taken when
+ * storing these numbers outside of Lucene. In particular, numbers may change in
+ * the following situations:</p>
+ * <ul>
+ * <li>
+ * <p>The numbers stored in each segment are unique only within the segment, and
+ * must be converted before they can be used in a larger context. The standard
+ * technique is to allocate each segment a range of values, based on the range of
+ * numbers used in that segment. To convert a document number from a segment to an
+ * external value, the segment's <i>base</i> document number is added. To convert
+ * an external value back to a segment-specific value, the segment is identified
+ * by the range that the external value is in, and the segment's base value is
+ * subtracted. For example two five document segments might be combined, so that
+ * the first segment has a base value of zero, and the second of five. Document
+ * three from the second segment would have an external value of eight.</p>
+ * </li>
+ * <li>
+ * <p>When documents are deleted, gaps are created in the numbering. These are
+ * eventually removed as the index evolves through merging. Deleted documents are
+ * dropped when segments are merged. A freshly-merged segment thus has no gaps in
+ * its numbering.</p>
+ * </li>
+ * </ul>
+ * </div>
+ * <a name="Overview"></a>
+ * <h2>Index Structure Overview</h2>
+ * <div>
+ * <p>Each segment index maintains the following:</p>
+ * <ul>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment info}.
+ *    This contains metadata about a segment, such as the number of documents,
+ *    what files it uses, 
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}. 
+ *    This contains the set of field names used in the index.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}. 
+ * This contains, for each document, a list of attribute-value pairs, where the attributes 
+ * are field names. These are used to store auxiliary information about the document, such as 
+ * its title, url, or an identifier to access a database. The set of stored fields are what is 
+ * returned for each hit when searching. This is keyed by document number.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}. 
+ * A dictionary containing all of the terms used in all of the
+ * indexed fields of all of the documents. The dictionary also contains the number
+ * of documents which contain the term, and pointers to the term's frequency and
+ * proximity data.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}. 
+ * For each term in the dictionary, the numbers of all the
+ * documents that contain that term, and the frequency of the term in that
+ * document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}. 
+ * For each term in the dictionary, the positions that the
+ * term occurs in each document. Note that this will not exist if all fields in
+ * all documents omit position data.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}. 
+ * For each field in each document, a value is stored
+ * that is multiplied into the score for hits on that field.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}. 
+ * For each field in each document, the term vector (sometimes
+ * called document vector) may be stored. A term vector consists of term text and
+ * term frequency. To add Term Vectors to your index see the 
+ * {@link org.apache.lucene.document.Field Field} constructors
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat Per-document values}. 
+ * Like stored values, these are also keyed by document
+ * number, but are generally intended to be loaded into main memory for fast
+ * access. Whereas stored values are generally intended for summary results from
+ * searches, per-document values are useful for things like scoring factors.
+ * </li>
+ * <li>
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}. 
+ * An optional file indicating which documents are live.
+ * </li>
+ * </ul>
+ * <p>Details on each of these are provided in their linked pages.</p>
+ * </div>
+ * <a name="File_Naming"></a>
+ * <h2>File Naming</h2>
+ * <div>
+ * <p>All files belonging to a segment have the same name with varying extensions.
+ * The extensions correspond to the different file formats described below. When
+ * using the Compound File format (default for small segments) these files (except
+ * for the Segment info file, the Lock file, and Deleted documents file) are collapsed 
+ * into a single .cfs file (see below for details)</p>
+ * <p>Typically, all segments in an index are stored in a single directory,
+ * although this is not required.</p>
+ * <p>File names are never re-used. That is, when any file is saved
+ * to the Directory it is given a never before used filename. This is achieved
+ * using a simple generations approach. For example, the first segments file is
+ * segments_1, then segments_2, etc. The generation is a sequential long integer
+ * represented in alpha-numeric (base 36) form.</p>
+ * </div>
+ * <a name="file-names"></a>
+ * <h2>Summary of File Extensions</h2>
+ * <div>
+ * <p>The following table summarizes the names and extensions of the files in
+ * Lucene:</p>
+ * <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
+ * <tr>
+ * <th>Name</th>
+ * <th>Extension</th>
+ * <th>Brief Description</th>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
+ * <td>segments_N</td>
+ * <td>Stores information about a commit point</td>
+ * </tr>
+ * <tr>
+ * <td><a href="#Lock_File">Lock File</a></td>
+ * <td>write.lock</td>
+ * <td>The Write lock prevents multiple IndexWriters from writing to the same
+ * file.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment Info}</td>
+ * <td>.si</td>
+ * <td>Stores metadata about a segment</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
+ * <td>.cfs, .cfe</td>
+ * <td>An optional "virtual" file consisting of all the other index files for
+ * systems that frequently run out of file handles.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
+ * <td>.fnm</td>
+ * <td>Stores information about the fields</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
+ * <td>.fdx</td>
+ * <td>Contains pointers to field data</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
+ * <td>.fdt</td>
+ * <td>The stored fields for documents</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
+ * <td>.tim</td>
+ * <td>The term dictionary, stores term info</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
+ * <td>.tip</td>
+ * <td>The index into the Term Dictionary</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
+ * <td>.doc</td>
+ * <td>Contains the list of docs which contain each term along with frequency</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
+ * <td>.pos</td>
+ * <td>Stores position information about where a term occurs in the index</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
+ * <td>.pay</td>
+ * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
+ * <td>.nvd, .nvm</td>
+ * <td>Encodes length and boost factors for docs and fields</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
+ * <td>.dvd, .dvm</td>
+ * <td>Encodes additional scoring factors or other per-document information.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
+ * <td>.tvx</td>
+ * <td>Stores offset into the document data file</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Data}</td>
+ * <td>.tvd</td>
+ * <td>Contains term vector data.</td>
+ * </tr>
+ * <tr>
+ * <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
+ * <td>.liv</td>
+ * <td>Info about what files are live</td>
+ * </tr>
+ * </table>
+ * </div>
+ * <a name="Lock_File"></a>
+ * <h2>Lock File</h2>
+ * The write lock, which is stored in the index directory by default, is named
+ * "write.lock". If the lock directory is different from the index directory then
+ * the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
+ * derived from the full path to the index directory. When this file is present, a
+ * writer is currently modifying the index (adding or removing documents). This
+ * lock file ensures that only one writer is modifying the index at a time.
+ * <a name="History"></a>
+ * <h2>History</h2>
+ * <p>Compatibility notes are provided in this document, describing how file
+ * formats have changed from prior versions:</p>
+ * <ul>
+ * <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
+ * no more commit lock). The change is fully backwards compatible: you can open a
+ * pre-2.1 index for searching or adding/deleting of docs. When the new segments
+ * file is saved (committed), it will be written in the new file format (meaning
+ * no specific "upgrade" process is needed). But note that once a commit has
+ * occurred, pre-2.1 Lucene will not be able to read the index.</li>
+ * <li>In version 2.3, the file format was changed to allow segments to share a
+ * single set of doc store (vectors &amp; stored fields) files. This allows for
+ * faster indexing in certain cases. The change is fully backwards compatible (in
+ * the same way as the lock-less commits change in 2.1).</li>
+ * <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
+ * Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
+ * LUCENE-510</a> for details.</li>
+ * <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
+ * may be passed to IndexWriter's commit methods (and later retrieved), which is
+ * recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
+ * LUCENE-1382</a> for details. Also,
+ * diagnostics were added to each segment written recording details about why it
+ * was written (due to flush, merge; which OS/JRE was used; etc.). See issue
+ * <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
+ * <li>In version 3.0, compressed fields are no longer written to the index (they
+ * can still be read, but on merge the new segment will write them, uncompressed).
+ * See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a> 
+ * for details.</li>
+ * <li>In version 3.1, segments records the code version that created them. See
+ * <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details. 
+ * Additionally segments track explicitly whether or not they have term vectors. 
+ * See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a> 
+ * for details.</li>
+ * <li>In version 3.2, numeric fields are written as natively to stored fields
+ * file, previously they were stored in text format only.</li>
+ * <li>In version 3.4, fields can omit position data while still indexing term
+ * frequencies.</li>
+ * <li>In version 4.0, the format of the inverted index became extensible via
+ * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
+ * ({@code DocValues}) was introduced. Normalization factors need no longer be a 
+ * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
+ * Terms need not be unicode strings, they can be any byte sequence. Term offsets 
+ * can optionally be indexed into the postings lists. Payloads can be stored in the 
+ * term vectors.</li>
+ * <li>In version 4.1, the format of the postings list changed to use either
+ * of FOR compression or variable-byte encoding, depending upon the frequency
+ * of the term. Terms appearing only once were changed to inline directly into
+ * the term dictionary. Stored fields are compressed by default. </li>
+ * <li>In version 4.2, term vectors are compressed by default. DocValues has 
+ * a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
+ * on multi-valued fields.</li>
+ * <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
+ * <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to 
+ * allow updating NumericDocValues fields.</li>
+ * <li>In version 4.8, checksum footers were added to the end of each index file 
+ * for improved data integrity. Specifically, the last 8 bytes of every index file
+ * contain the zlib-crc32 checksum of the file.</li>
+ * <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
+ * that is suitable for faceting/sorting/analytics.
+ * <li>In version 5.4, DocValues have been improved to store more information on disk:
+ * addresses for binary fields and ord indexes for multi-valued fields.
+ * <li>In version 7.0, DocValues have been improved to better support sparse doc values
+ * thanks to an iterator API.
+ * </li>
+ * </ul>
+ * <a name="Limitations"></a>
+ * <h2>Limitations</h2>
+ * <div>
+ * <p>Lucene uses a Java <code>int</code> to refer to
+ * document numbers, and the index file format uses an <code>Int32</code>
+ * on-disk to store document numbers. This is a limitation
+ * of both the index file format and the current implementation. Eventually these
+ * should be replaced with either <code>UInt64</code> values, or
+ * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
+ * </div>
+ */
+package org.apache.lucene.codecs.lucene70;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
----------------------------------------------------------------------
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 548f8d0..85aa3a7 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene62.Lucene62Codec
+org.apache.lucene.codecs.lucene70.Lucene70Codec

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
----------------------------------------------------------------------
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
index 26984ef..1161f03 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat
+org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
index f945c2d..909f658 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormatHighCompression.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene50;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@@ -33,7 +33,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
   @Override
   protected Codec getCodec() {
-    return new Lucene62Codec(Mode.BEST_COMPRESSION);
+    return new Lucene70Codec(Mode.BEST_COMPRESSION);
   }
   
   /**
@@ -44,7 +44,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
     Directory dir = newDirectory();
     for (int i = 0; i < 10; i++) {
       IndexWriterConfig iwc = newIndexWriterConfig();
-      iwc.setCodec(new Lucene62Codec(RandomPicks.randomFrom(random(), Mode.values())));
+      iwc.setCodec(new Lucene70Codec(RandomPicks.randomFrom(random(), Mode.values())));
       IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
       Document doc = new Document();
       doc.add(new StoredField("field1", "value1"));
@@ -71,7 +71,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
   
   public void testInvalidOptions() throws Exception {
     expectThrows(NullPointerException.class, () -> {
-      new Lucene62Codec(null);
+      new Lucene70Codec(null);
     });
     
     expectThrows(NullPointerException.class, () -> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
index a0ad87f..c87c51f 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
@@ -18,14 +18,14 @@ package org.apache.lucene.codecs.lucene53;
 
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.lucene.index.BaseNormsFormatTestCase;
 
 /**
  * Tests Lucene53NormsFormat
  */
 public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
-  private final Codec codec = new Lucene62Codec();
+  private final Codec codec = new Lucene70Codec();
   
   @Override
   protected Codec getCodec() {