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

[lucene-solr] branch master updated: LUCENE-9669: Restore Lucene70Codec (#2221)

This is an automated email from the ASF dual-hosted git repository.

simonw pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 1aeb1dc  LUCENE-9669: Restore Lucene70Codec (#2221)
1aeb1dc is described below

commit 1aeb1dcb86904f2cd3799bb467a56986d97f0d6e
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Tue Jan 19 21:07:54 2021 +0100

    LUCENE-9669: Restore Lucene70Codec (#2221)
    
    This change restores the Lucene70Codec for file format compatibility of
    indices that are created within the Lucene 7 major version. These indices
    can be opened via an expert API on DirectoryReader in read-only mode. Changes
    to these indices are prohibited and will be rejected by the IndexWriter.
    In fact, IndexWriter will not open an index that is created with a major version
    less than N-1 to the current major version.
---
 lucene/CHANGES.txt                                 |    9 +
 .../backward_codecs/lucene70/IndexedDISI.java      |  326 +++++
 .../backward_codecs/lucene70/Lucene70Codec.java    |  140 ++
 .../lucene70/Lucene70DocValuesConsumer.java        |  703 ++++++++++
 .../lucene70/Lucene70DocValuesFormat.java          |  171 +++
 .../lucene70/Lucene70DocValuesProducer.java        | 1478 ++++++++++++++++++++
 .../lucene70/Lucene70NormsConsumer.java            |  169 +++
 .../lucene70/Lucene70NormsFormat.java              |  106 ++
 .../lucene70/Lucene70NormsProducer.java            |  464 ++++++
 .../services/org.apache.lucene.codecs.Codec        |    1 +
 .../org.apache.lucene.codecs.DocValuesFormat       |    1 +
 .../backward_codecs/lucene70/Lucene70RWCodec.java  |   55 +
 .../backward_codecs/lucene70/TestIndexedDISI.java  |  248 ++++
 .../lucene70/TestLucene70DocValuesFormat.java      |  694 +++++++++
 .../lucene70/TestLucene70NormsFormat.java          |   28 +
 .../backward_index/TestBackwardsCompatibility.java |   64 +-
 16 files changed, 4630 insertions(+), 27 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 2bdd5d4..92c0c32 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -197,6 +197,15 @@ Bug fixes
 * LUCENE-9365: FuzzyQuery was missing matches when prefix length was equal to the term length
   (Mark Harwood, Mike Drob)
 
+Changes in Backwards Compatibility Policy
+
+* LUCENE-9669: DirectoryReader#open now accepts an argument to open indices created with versions
+  older than N-1. Lucene now can open indices created with a major version of N-2 in read-only mode.
+  Opening an index created with a major version of N-2 with an IndexWriter is not supported.
+  Further does lucene only support file-format compatibilty which enables reading of old indices while
+  semantic changes like analysis or certain encoding on top of the file format are only supported on 
+  a best effort basis. (Simon Willnauer)  
+
 Other
 
 * LUCENE-9570, LUCENE-9564: Apply google java format and enforce it on source Java files.
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/IndexedDISI.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/IndexedDISI.java
new file mode 100644
index 0000000..fcefdd6
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/IndexedDISI.java
@@ -0,0 +1,326 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import java.io.DataInput;
+import java.io.IOException;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.RoaringDocIdSet;
+
+/**
+ * Disk-based implementation of a {@link DocIdSetIterator} which can return the index of the current
+ * document, i.e. the ordinal of the current document among the list of documents that this iterator
+ * can return. This is useful to implement sparse doc values by only having to encode values for
+ * documents that actually have a value.
+ *
+ * <p>Implementation-wise, this {@link DocIdSetIterator} is inspired of {@link RoaringDocIdSet
+ * roaring bitmaps} and encodes ranges of {@code 65536} documents independently and picks between 3
+ * encodings depending on the density of the range:
+ *
+ * <ul>
+ *   <li>{@code ALL} if the range contains 65536 documents exactly,
+ *   <li>{@code DENSE} if the range contains 4096 documents or more; in that case documents are
+ *       stored in a bit set,
+ *   <li>{@code SPARSE} otherwise, and the lower 16 bits of the doc IDs are stored in a {@link
+ *       DataInput#readShort() short}.
+ * </ul>
+ *
+ * <p>Only ranges that contain at least one value are encoded.
+ *
+ * <p>This implementation uses 6 bytes per document in the worst-case, which happens in the case
+ * that all ranges contain exactly one document.
+ *
+ * @lucene.internal
+ */
+final class IndexedDISI extends DocIdSetIterator {
+
+  static final int MAX_ARRAY_LENGTH = (1 << 12) - 1;
+
+  private static void flush(int block, FixedBitSet buffer, int cardinality, IndexOutput out)
+      throws IOException {
+    assert block >= 0 && block < 65536;
+    out.writeShort((short) block);
+    assert cardinality > 0 && cardinality <= 65536;
+    out.writeShort((short) (cardinality - 1));
+    if (cardinality > MAX_ARRAY_LENGTH) {
+      if (cardinality != 65536) { // all docs are set
+        for (long word : buffer.getBits()) {
+          out.writeLong(word);
+        }
+      }
+    } else {
+      BitSetIterator it = new BitSetIterator(buffer, cardinality);
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        out.writeShort((short) doc);
+      }
+    }
+  }
+
+  static void writeBitSet(DocIdSetIterator it, IndexOutput out) throws IOException {
+    int i = 0;
+    final FixedBitSet buffer = new FixedBitSet(1 << 16);
+    int prevBlock = -1;
+    for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+      final int block = doc >>> 16;
+      if (prevBlock != -1 && block != prevBlock) {
+        flush(prevBlock, buffer, i, out);
+        buffer.clear(0, buffer.length());
+        prevBlock = block;
+        i = 0;
+      }
+      buffer.set(doc & 0xFFFF);
+      i++;
+      prevBlock = block;
+    }
+    if (i > 0) {
+      flush(prevBlock, buffer, i, out);
+      buffer.clear(0, buffer.length());
+    }
+    // NO_MORE_DOCS is stored explicitly
+    buffer.set(DocIdSetIterator.NO_MORE_DOCS & 0xFFFF);
+    flush(DocIdSetIterator.NO_MORE_DOCS >>> 16, buffer, 1, out);
+  }
+
+  /** The slice that stores the {@link DocIdSetIterator}. */
+  private final IndexInput slice;
+
+  private final long cost;
+
+  IndexedDISI(IndexInput in, long offset, long length, long cost) throws IOException {
+    this(in.slice("docs", offset, length), cost);
+  }
+
+  // This constructor allows to pass the slice directly in case it helps reuse
+  // see eg. Lucene70 norms producer's merge instance
+  IndexedDISI(IndexInput slice, long cost) throws IOException {
+    this.slice = slice;
+    this.cost = cost;
+  }
+
+  private int block = -1;
+  private long blockEnd;
+  private int nextBlockIndex = -1;
+  Method method;
+
+  private int doc = -1;
+  private int index = -1;
+
+  // SPARSE variables
+  boolean exists;
+
+  // DENSE variables
+  private long word;
+  private int wordIndex = -1;
+  // number of one bits encountered so far, including those of `word`
+  private int numberOfOnes;
+
+  // ALL variables
+  private int gap;
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    final int targetBlock = target & 0xFFFF0000;
+    if (block < targetBlock) {
+      advanceBlock(targetBlock);
+    }
+    if (block == targetBlock) {
+      if (method.advanceWithinBlock(this, target)) {
+        return doc;
+      }
+      readBlockHeader();
+    }
+    boolean found = method.advanceWithinBlock(this, block);
+    assert found;
+    return doc;
+  }
+
+  public boolean advanceExact(int target) throws IOException {
+    final int targetBlock = target & 0xFFFF0000;
+    if (block < targetBlock) {
+      advanceBlock(targetBlock);
+    }
+    boolean found = block == targetBlock && method.advanceExactWithinBlock(this, target);
+    this.doc = target;
+    return found;
+  }
+
+  private void advanceBlock(int targetBlock) throws IOException {
+    do {
+      slice.seek(blockEnd);
+      readBlockHeader();
+    } while (block < targetBlock);
+  }
+
+  private void readBlockHeader() throws IOException {
+    block = Short.toUnsignedInt(slice.readShort()) << 16;
+    assert block >= 0;
+    final int numValues = 1 + Short.toUnsignedInt(slice.readShort());
+    index = nextBlockIndex;
+    nextBlockIndex = index + numValues;
+    if (numValues <= MAX_ARRAY_LENGTH) {
+      method = Method.SPARSE;
+      blockEnd = slice.getFilePointer() + (numValues << 1);
+    } else if (numValues == 65536) {
+      method = Method.ALL;
+      blockEnd = slice.getFilePointer();
+      gap = block - index - 1;
+    } else {
+      method = Method.DENSE;
+      blockEnd = slice.getFilePointer() + (1 << 13);
+      wordIndex = -1;
+      numberOfOnes = index + 1;
+    }
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return advance(doc + 1);
+  }
+
+  public int index() {
+    return index;
+  }
+
+  @Override
+  public long cost() {
+    return cost;
+  }
+
+  enum Method {
+    SPARSE {
+      @Override
+      boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException {
+        final int targetInBlock = target & 0xFFFF;
+        // TODO: binary search
+        for (; disi.index < disi.nextBlockIndex; ) {
+          int doc = Short.toUnsignedInt(disi.slice.readShort());
+          disi.index++;
+          if (doc >= targetInBlock) {
+            disi.doc = disi.block | doc;
+            disi.exists = true;
+            return true;
+          }
+        }
+        return false;
+      }
+
+      @Override
+      boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException {
+        final int targetInBlock = target & 0xFFFF;
+        // TODO: binary search
+        if (target == disi.doc) {
+          return disi.exists;
+        }
+        for (; disi.index < disi.nextBlockIndex; ) {
+          int doc = Short.toUnsignedInt(disi.slice.readShort());
+          disi.index++;
+          if (doc >= targetInBlock) {
+            if (doc != targetInBlock) {
+              disi.index--;
+              disi.slice.seek(disi.slice.getFilePointer() - Short.BYTES);
+              break;
+            }
+            disi.exists = true;
+            return true;
+          }
+        }
+        disi.exists = false;
+        return false;
+      }
+    },
+    DENSE {
+      @Override
+      boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException {
+        final int targetInBlock = target & 0xFFFF;
+        final int targetWordIndex = targetInBlock >>> 6;
+        for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) {
+          disi.word = disi.slice.readLong();
+          disi.numberOfOnes += Long.bitCount(disi.word);
+        }
+        disi.wordIndex = targetWordIndex;
+
+        long leftBits = disi.word >>> target;
+        if (leftBits != 0L) {
+          disi.doc = target + Long.numberOfTrailingZeros(leftBits);
+          disi.index = disi.numberOfOnes - Long.bitCount(leftBits);
+          return true;
+        }
+
+        while (++disi.wordIndex < 1024) {
+          disi.word = disi.slice.readLong();
+          if (disi.word != 0) {
+            disi.index = disi.numberOfOnes;
+            disi.numberOfOnes += Long.bitCount(disi.word);
+            disi.doc = disi.block | (disi.wordIndex << 6) | Long.numberOfTrailingZeros(disi.word);
+            return true;
+          }
+        }
+        return false;
+      }
+
+      @Override
+      boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException {
+        final int targetInBlock = target & 0xFFFF;
+        final int targetWordIndex = targetInBlock >>> 6;
+        for (int i = disi.wordIndex + 1; i <= targetWordIndex; ++i) {
+          disi.word = disi.slice.readLong();
+          disi.numberOfOnes += Long.bitCount(disi.word);
+        }
+        disi.wordIndex = targetWordIndex;
+
+        long leftBits = disi.word >>> target;
+        disi.index = disi.numberOfOnes - Long.bitCount(leftBits);
+        return (leftBits & 1L) != 0;
+      }
+    },
+    ALL {
+      @Override
+      boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException {
+        disi.doc = target;
+        disi.index = target - disi.gap;
+        return true;
+      }
+
+      @Override
+      boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException {
+        disi.index = target - disi.gap;
+        return true;
+      }
+    };
+
+    /**
+     * Advance to the first doc from the block that is equal to or greater than {@code target}.
+     * Return true if there is such a doc and false otherwise.
+     */
+    abstract boolean advanceWithinBlock(IndexedDISI disi, int target) throws IOException;
+
+    /**
+     * Advance the iterator exactly to the position corresponding to the given {@code target} and
+     * return whether this document exists.
+     */
+    abstract boolean advanceExactWithinBlock(IndexedDISI disi, int target) throws IOException;
+  }
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70Codec.java
new file mode 100644
index 0000000..b1ee4b4
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70Codec.java
@@ -0,0 +1,140 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.backward_codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.backward_codecs.lucene60.Lucene60PointsFormat;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.VectorFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 7.0 index format, with configurable per-field postings and docvalues
+ * formats.
+ *
+ * <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.backward_codecs.lucene70 package documentation for file format details.
+ * @lucene.experimental
+ */
+public class Lucene70Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene70SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
+  private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
+
+  private final PostingsFormat postingsFormat =
+      new PerFieldPostingsFormat() {
+        @Override
+        public PostingsFormat getPostingsFormatForField(String field) {
+          throw new IllegalStateException(
+              "This codec should only be used for reading, not writing");
+        }
+      };
+
+  private final DocValuesFormat docValuesFormat =
+      new PerFieldDocValuesFormat() {
+        @Override
+        public DocValuesFormat getDocValuesFormatForField(String field) {
+          return defaultDVFormat;
+        }
+      };
+
+  private final StoredFieldsFormat storedFieldsFormat =
+      new Lucene50StoredFieldsFormat(Mode.BEST_SPEED);
+
+  /** Instantiates a new codec. */
+  public Lucene70Codec() {
+    super("Lucene70");
+  }
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFieldsFormat;
+  }
+
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+
+  @Override
+  public SegmentInfoFormat segmentInfoFormat() {
+    return segmentInfosFormat;
+  }
+
+  @Override
+  public final LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+  @Override
+  public final CompoundFormat compoundFormat() {
+    return compoundFormat;
+  }
+
+  @Override
+  public final PointsFormat pointsFormat() {
+    return new Lucene60PointsFormat();
+  }
+
+  @Override
+  public VectorFormat vectorFormat() {
+    return VectorFormat.EMPTY;
+  }
+
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  private final NormsFormat normsFormat = new Lucene70NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesConsumer.java
new file mode 100644
index 0000000..a8c1bad
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesConsumer.java
@@ -0,0 +1,703 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SHIFT;
+import static org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.EmptyDocValuesProducer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+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.search.DocIdSetIterator;
+import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+import org.apache.lucene.util.packed.DirectWriter;
+
+/** writer for {@link Lucene70DocValuesFormat} */
+final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+  IndexOutput data, meta;
+  final int maxDoc;
+
+  /** expert: Creates a new writer */
+  public Lucene70DocValuesConsumer(
+      SegmentWriteState state,
+      String dataCodec,
+      String dataExtension,
+      String metaCodec,
+      String metaExtension)
+      throws IOException {
+    boolean success = false;
+    try {
+      String dataName =
+          IndexFileNames.segmentFileName(
+              state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeIndexHeader(
+          data,
+          dataCodec,
+          Lucene70DocValuesFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      String metaName =
+          IndexFileNames.segmentFileName(
+              state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeIndexHeader(
+          meta,
+          metaCodec,
+          Lucene70DocValuesFormat.VERSION_CURRENT,
+          state.segmentInfo.getId(),
+          state.segmentSuffix);
+      maxDoc = state.segmentInfo.maxDoc();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeInt(-1); // write EOF marker
+        CodecUtil.writeFooter(meta); // write checksum
+      }
+      if (data != null) {
+        CodecUtil.writeFooter(data); // write checksum
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+      meta = data = null;
+    }
+  }
+
+  @Override
+  public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer)
+      throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
+
+    writeValues(
+        field,
+        new EmptyDocValuesProducer() {
+          @Override
+          public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+            return DocValues.singleton(valuesProducer.getNumeric(field));
+          }
+        });
+  }
+
+  private static class MinMaxTracker {
+    long min, max, numValues, spaceInBits;
+
+    MinMaxTracker() {
+      reset();
+      spaceInBits = 0;
+    }
+
+    private void reset() {
+      min = Long.MAX_VALUE;
+      max = Long.MIN_VALUE;
+      numValues = 0;
+    }
+
+    /** Accumulate a new value. */
+    void update(long v) {
+      min = Math.min(min, v);
+      max = Math.max(max, v);
+      ++numValues;
+    }
+
+    /** Update the required space. */
+    void finish() {
+      if (max > min) {
+        spaceInBits += DirectWriter.unsignedBitsRequired(max - min) * numValues;
+      }
+    }
+
+    /** Update space usage and get ready for accumulating values for the next block. */
+    void nextBlock() {
+      finish();
+      reset();
+    }
+  }
+
+  private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
+    int numDocsWithValue = 0;
+    MinMaxTracker minMax = new MinMaxTracker();
+    MinMaxTracker blockMinMax = new MinMaxTracker();
+    long gcd = 0;
+    Set<Long> uniqueValues = new HashSet<>();
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        long v = values.nextValue();
+
+        if (gcd != 1) {
+          if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
+            // in that case v - minValue might overflow and make the GCD computation return
+            // wrong results. Since these extreme values are unlikely, we just discard
+            // GCD computation for them
+            gcd = 1;
+          } else if (minMax.numValues != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minMax.min);
+          }
+        }
+
+        minMax.update(v);
+        blockMinMax.update(v);
+        if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) {
+          blockMinMax.nextBlock();
+        }
+
+        if (uniqueValues != null && uniqueValues.add(v) && uniqueValues.size() > 256) {
+          uniqueValues = null;
+        }
+      }
+
+      numDocsWithValue++;
+    }
+
+    minMax.finish();
+    blockMinMax.finish();
+
+    final long numValues = minMax.numValues;
+    long min = minMax.min;
+    final long max = minMax.max;
+    assert blockMinMax.spaceInBits <= minMax.spaceInBits;
+
+    if (numDocsWithValue == 0) {
+      meta.writeLong(-2);
+      meta.writeLong(0L);
+    } else if (numDocsWithValue == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
+    } else {
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getSortedNumeric(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
+    }
+
+    meta.writeLong(numValues);
+    final int numBitsPerValue;
+    boolean doBlocks = false;
+    Map<Long, Integer> encode = null;
+    if (min >= max) {
+      numBitsPerValue = 0;
+      meta.writeInt(-1);
+    } else {
+      if (uniqueValues != null
+          && uniqueValues.size() > 1
+          && DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1)
+              < DirectWriter.unsignedBitsRequired((max - min) / gcd)) {
+        numBitsPerValue = DirectWriter.unsignedBitsRequired(uniqueValues.size() - 1);
+        final Long[] sortedUniqueValues = uniqueValues.toArray(new Long[0]);
+        Arrays.sort(sortedUniqueValues);
+        meta.writeInt(sortedUniqueValues.length);
+        for (Long v : sortedUniqueValues) {
+          meta.writeLong(v);
+        }
+        encode = new HashMap<>();
+        for (int i = 0; i < sortedUniqueValues.length; ++i) {
+          encode.put(sortedUniqueValues[i], i);
+        }
+        min = 0;
+        gcd = 1;
+      } else {
+        uniqueValues = null;
+        // we do blocks if that appears to save 10+% storage
+        doBlocks =
+            minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9;
+        if (doBlocks) {
+          numBitsPerValue = 0xFF;
+          meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT);
+        } else {
+          numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
+          if (gcd == 1
+              && min > 0
+              && DirectWriter.unsignedBitsRequired(max)
+                  == DirectWriter.unsignedBitsRequired(max - min)) {
+            min = 0;
+          }
+          meta.writeInt(-1);
+        }
+      }
+    }
+
+    meta.writeByte((byte) numBitsPerValue);
+    meta.writeLong(min);
+    meta.writeLong(gcd);
+    long startOffset = data.getFilePointer();
+    meta.writeLong(startOffset);
+    if (doBlocks) {
+      writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd);
+    } else if (numBitsPerValue != 0) {
+      writeValuesSingleBlock(
+          valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode);
+    }
+    meta.writeLong(data.getFilePointer() - startOffset);
+
+    return new long[] {numDocsWithValue, numValues};
+  }
+
+  private void writeValuesSingleBlock(
+      SortedNumericDocValues values,
+      long numValues,
+      int numBitsPerValue,
+      long min,
+      long gcd,
+      Map<Long, Integer> encode)
+      throws IOException {
+    DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        long v = values.nextValue();
+        if (encode == null) {
+          writer.add((v - min) / gcd);
+        } else {
+          writer.add(encode.get(v));
+        }
+      }
+    }
+    writer.finish();
+  }
+
+  private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd)
+      throws IOException {
+    final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
+    final ByteBuffersDataOutput encodeBuffer = ByteBuffersDataOutput.newResettableInstance();
+    int upTo = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        buffer[upTo++] = values.nextValue();
+        if (upTo == NUMERIC_BLOCK_SIZE) {
+          writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer);
+          upTo = 0;
+        }
+      }
+    }
+    if (upTo > 0) {
+      writeBlock(buffer, upTo, gcd, encodeBuffer);
+    }
+  }
+
+  private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutput buffer)
+      throws IOException {
+    assert length > 0;
+    long min = values[0];
+    long max = values[0];
+    for (int i = 1; i < length; ++i) {
+      final long v = values[i];
+      assert Math.floorMod(values[i] - min, gcd) == 0;
+      min = Math.min(min, v);
+      max = Math.max(max, v);
+    }
+    if (min == max) {
+      data.writeByte((byte) 0);
+      data.writeLong(min);
+    } else {
+      final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min);
+      buffer.reset();
+      assert buffer.size() == 0;
+      final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue);
+      for (int i = 0; i < length; ++i) {
+        w.add((values[i] - min) / gcd);
+      }
+      w.finish();
+      data.writeByte((byte) bitsPerValue);
+      data.writeLong(min);
+      data.writeInt(Math.toIntExact(buffer.size()));
+      buffer.copyTo(data);
+    }
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.BINARY);
+
+    BinaryDocValues values = valuesProducer.getBinary(field);
+    long start = data.getFilePointer();
+    meta.writeLong(start);
+    int numDocsWithField = 0;
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      numDocsWithField++;
+      BytesRef v = values.binaryValue();
+      int length = v.length;
+      data.writeBytes(v.bytes, v.offset, v.length);
+      minLength = Math.min(length, minLength);
+      maxLength = Math.max(length, maxLength);
+    }
+    assert numDocsWithField <= maxDoc;
+    meta.writeLong(data.getFilePointer() - start);
+
+    if (numDocsWithField == 0) {
+      meta.writeLong(-2);
+      meta.writeLong(0L);
+    } else if (numDocsWithField == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
+    } else {
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getBinary(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
+    }
+
+    meta.writeInt(numDocsWithField);
+    meta.writeInt(minLength);
+    meta.writeInt(maxLength);
+    if (maxLength > minLength) {
+      start = data.getFilePointer();
+      meta.writeLong(start);
+      meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+      final DirectMonotonicWriter writer =
+          DirectMonotonicWriter.getInstance(
+              meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+      long addr = 0;
+      writer.add(addr);
+      values = valuesProducer.getBinary(field);
+      for (int doc = values.nextDoc();
+          doc != DocIdSetIterator.NO_MORE_DOCS;
+          doc = values.nextDoc()) {
+        addr += values.binaryValue().length;
+        writer.add(addr);
+      }
+      writer.finish();
+      meta.writeLong(data.getFilePointer() - start);
+    }
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.SORTED);
+    doAddSortedField(field, valuesProducer);
+  }
+
+  private void doAddSortedField(FieldInfo field, DocValuesProducer valuesProducer)
+      throws IOException {
+    SortedDocValues values = valuesProducer.getSorted(field);
+    int numDocsWithField = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      numDocsWithField++;
+    }
+
+    if (numDocsWithField == 0) {
+      meta.writeLong(-2);
+      meta.writeLong(0L);
+    } else if (numDocsWithField == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
+    } else {
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getSorted(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
+    }
+
+    meta.writeInt(numDocsWithField);
+    if (values.getValueCount() <= 1) {
+      meta.writeByte((byte) 0);
+      meta.writeLong(0L);
+      meta.writeLong(0L);
+    } else {
+      int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
+      meta.writeByte((byte) numberOfBitsPerOrd);
+      long start = data.getFilePointer();
+      meta.writeLong(start);
+      DirectWriter writer = DirectWriter.getInstance(data, numDocsWithField, numberOfBitsPerOrd);
+      values = valuesProducer.getSorted(field);
+      for (int doc = values.nextDoc();
+          doc != DocIdSetIterator.NO_MORE_DOCS;
+          doc = values.nextDoc()) {
+        writer.add(values.ordValue());
+      }
+      writer.finish();
+      meta.writeLong(data.getFilePointer() - start);
+    }
+
+    addTermsDict(DocValues.singleton(valuesProducer.getSorted(field)));
+  }
+
+  private void addTermsDict(SortedSetDocValues values) throws IOException {
+    final long size = values.getValueCount();
+    meta.writeVLong(size);
+    meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT);
+
+    ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
+    ByteBuffersIndexOutput addressIndexOut =
+        new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
+    meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long numBlocks =
+        (size + Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK)
+            >>> Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
+    DirectMonotonicWriter writer =
+        DirectMonotonicWriter.getInstance(
+            meta, addressIndexOut, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    BytesRefBuilder previous = new BytesRefBuilder();
+    long ord = 0;
+    long start = data.getFilePointer();
+    int maxLength = 0;
+    TermsEnum iterator = values.termsEnum();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if ((ord & Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK) == 0) {
+        writer.add(data.getFilePointer() - start);
+        data.writeVInt(term.length);
+        data.writeBytes(term.bytes, term.offset, term.length);
+      } else {
+        final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
+        final int suffixLength = term.length - prefixLength;
+        assert suffixLength > 0; // terms are unique
+
+        data.writeByte((byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
+        if (prefixLength >= 15) {
+          data.writeVInt(prefixLength - 15);
+        }
+        if (suffixLength >= 16) {
+          data.writeVInt(suffixLength - 16);
+        }
+        data.writeBytes(term.bytes, term.offset + prefixLength, term.length - prefixLength);
+      }
+      maxLength = Math.max(maxLength, term.length);
+      previous.copyBytes(term);
+      ++ord;
+    }
+    writer.finish();
+    meta.writeInt(maxLength);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+    start = data.getFilePointer();
+    addressBuffer.copyTo(data);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+
+    // Now write the reverse terms index
+    writeTermsIndex(values);
+  }
+
+  private void writeTermsIndex(SortedSetDocValues values) throws IOException {
+    final long size = values.getValueCount();
+    meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
+    long start = data.getFilePointer();
+
+    long numBlocks =
+        1L
+            + ((size + Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK)
+                >>> Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
+    ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
+    ByteBuffersIndexOutput addressIndexOut =
+        new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
+    DirectMonotonicWriter writer =
+        DirectMonotonicWriter.getInstance(
+            meta, addressIndexOut, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    TermsEnum iterator = values.termsEnum();
+    BytesRefBuilder previous = new BytesRefBuilder();
+    long offset = 0;
+    long ord = 0;
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) == 0) {
+        writer.add(offset);
+        final int sortKeyLength;
+        if (ord == 0) {
+          // no previous term: no bytes to write
+          sortKeyLength = 0;
+        } else {
+          sortKeyLength = StringHelper.sortKeyLength(previous.get(), term);
+        }
+        offset += sortKeyLength;
+        data.writeBytes(term.bytes, term.offset, sortKeyLength);
+      } else if ((ord & Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK)
+          == Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) {
+        previous.copyBytes(term);
+      }
+      ++ord;
+    }
+    writer.add(offset);
+    writer.finish();
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+    start = data.getFilePointer();
+    addressBuffer.copyTo(data);
+    meta.writeLong(start);
+    meta.writeLong(data.getFilePointer() - start);
+  }
+
+  @Override
+  public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer)
+      throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.SORTED_NUMERIC);
+
+    long[] stats = writeValues(field, valuesProducer);
+    int numDocsWithField = Math.toIntExact(stats[0]);
+    long numValues = stats[1];
+    assert numValues >= numDocsWithField;
+
+    meta.writeInt(numDocsWithField);
+    if (numValues > numDocsWithField) {
+      long start = data.getFilePointer();
+      meta.writeLong(start);
+      meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+      final DirectMonotonicWriter addressesWriter =
+          DirectMonotonicWriter.getInstance(
+              meta, data, numDocsWithField + 1L, DIRECT_MONOTONIC_BLOCK_SHIFT);
+      long addr = 0;
+      addressesWriter.add(addr);
+      SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
+      for (int doc = values.nextDoc();
+          doc != DocIdSetIterator.NO_MORE_DOCS;
+          doc = values.nextDoc()) {
+        addr += values.docValueCount();
+        addressesWriter.add(addr);
+      }
+      addressesWriter.finish();
+      meta.writeLong(data.getFilePointer() - start);
+    }
+  }
+
+  @Override
+  public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer)
+      throws IOException {
+    meta.writeInt(field.number);
+    meta.writeByte(Lucene70DocValuesFormat.SORTED_SET);
+
+    SortedSetDocValues values = valuesProducer.getSortedSet(field);
+    int numDocsWithField = 0;
+    long numOrds = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      numDocsWithField++;
+      for (long ord = values.nextOrd();
+          ord != SortedSetDocValues.NO_MORE_ORDS;
+          ord = values.nextOrd()) {
+        numOrds++;
+      }
+    }
+
+    if (numDocsWithField == numOrds) {
+      meta.writeByte((byte) 0);
+      doAddSortedField(
+          field,
+          new EmptyDocValuesProducer() {
+            @Override
+            public SortedDocValues getSorted(FieldInfo field) throws IOException {
+              return SortedSetSelector.wrap(
+                  valuesProducer.getSortedSet(field), SortedSetSelector.Type.MIN);
+            }
+          });
+      return;
+    }
+    meta.writeByte((byte) 1);
+
+    assert numDocsWithField != 0;
+    if (numDocsWithField == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
+    } else {
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = valuesProducer.getSortedSet(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
+    }
+
+    int numberOfBitsPerOrd = DirectWriter.unsignedBitsRequired(values.getValueCount() - 1);
+    meta.writeByte((byte) numberOfBitsPerOrd);
+    long start = data.getFilePointer();
+    meta.writeLong(start);
+    DirectWriter writer = DirectWriter.getInstance(data, numOrds, numberOfBitsPerOrd);
+    values = valuesProducer.getSortedSet(field);
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (long ord = values.nextOrd();
+          ord != SortedSetDocValues.NO_MORE_ORDS;
+          ord = values.nextOrd()) {
+        writer.add(ord);
+      }
+    }
+    writer.finish();
+    meta.writeLong(data.getFilePointer() - start);
+
+    meta.writeInt(numDocsWithField);
+    start = data.getFilePointer();
+    meta.writeLong(start);
+    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    final DirectMonotonicWriter addressesWriter =
+        DirectMonotonicWriter.getInstance(
+            meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long addr = 0;
+    addressesWriter.add(addr);
+    values = valuesProducer.getSortedSet(field);
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      values.nextOrd();
+      addr++;
+      while (values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
+        addr++;
+      }
+      addressesWriter.add(addr);
+    }
+    addressesWriter.finish();
+    meta.writeLong(data.getFilePointer() - start);
+
+    addTermsDict(values);
+  }
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesFormat.java
new file mode 100644
index 0000000..8cc2517
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesFormat.java
@@ -0,0 +1,171 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.packed.DirectWriter;
+
+/**
+ * Lucene 7.0 DocValues format.
+ *
+ * <p>Documents that have a value for the field are encoded in a way that it is always possible to
+ * know the ordinal of the current document in the set of documents that have a value. For instance,
+ * say the set of documents that have a value for the field is <code>{1, 5, 6, 11}</code>. When the
+ * iterator is on <code>6</code>, it knows that this is the 3rd item of the set. This way, values
+ * can be stored densely and accessed based on their index at search time. If all documents in a
+ * segment have a value for the field, the index is the same as the doc ID, so this case is encoded
+ * implicitly and is very fast at query time. On the other hand if some documents are missing a
+ * value for the field then the set of documents that have a value is encoded into blocks. All doc
+ * IDs that share the same upper 16 bits are encoded into the same block with the following
+ * strategies:
+ *
+ * <ul>
+ *   <li>SPARSE: This strategy is used when a block contains at most 4095 documents. The lower 16
+ *       bits of doc IDs are stored as {@link DataOutput#writeShort(short) shorts} while the upper
+ *       16 bits are given by the block ID.
+ *   <li>DENSE: This strategy is used when a block contains between 4096 and 65535 documents. The
+ *       lower bits of doc IDs are stored in a bit set. Advancing is performed using {@link
+ *       Long#numberOfTrailingZeros(long) ntz} operations while the index is computed by
+ *       accumulating the {@link Long#bitCount(long) bit counts} of the visited longs.
+ *   <li>ALL: This strategy is used when a block contains exactly 65536 documents, meaning that the
+ *       block is full. In that case doc IDs do not need to be stored explicitly. This is typically
+ *       faster than both SPARSE and DENSE which is a reason why it is preferable to have all
+ *       documents that have a value for a field using contiguous doc IDs, for instance by using
+ *       {@link IndexWriterConfig#setIndexSort(org.apache.lucene.search.Sort) index sorting}.
+ * </ul>
+ *
+ * <p>Then the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) are
+ * encoded using the following strategies:
+ *
+ * <p>{@link DocValuesType#NUMERIC NUMERIC}:
+ *
+ * <ul>
+ *   <li>Delta-compressed: per-document integers written as deltas from the minimum value,
+ *       compressed with bitpacking. For more information, see {@link DirectWriter}.
+ *   <li>Table-compressed: when the number of unique values is very small (&lt; 256), and when there
+ *       are unused "gaps" in the range of values used (such as {@link SmallFloat}), a lookup table
+ *       is written instead. Each per-document entry is instead the ordinal to this table, and those
+ *       ordinals are compressed with bitpacking ({@link DirectWriter}).
+ *   <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest common
+ *       denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
+ *   <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are
+ *       written as blocks of bitpacked integers, encoding the deviation from the expected delta.
+ *   <li>Const-compressed: when there is only one possible value, no per-document data is needed and
+ *       this value is encoded alone.
+ * </ul>
+ *
+ * <p>{@link DocValuesType#BINARY BINARY}:
+ *
+ * <ul>
+ *   <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
+ *       Each document's value can be addressed directly with multiplication ({@code docID *
+ *       length}).
+ *   <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
+ *       for each document. The addresses are written as Monotonic-compressed numerics.
+ *   <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
+ *       completely and other values sharing prefixes. chunk addresses are written as
+ *       Monotonic-compressed numerics. A reverse lookup index is written from a portion of every
+ *       1024th term.
+ * </ul>
+ *
+ * <p>{@link DocValuesType#SORTED SORTED}:
+ *
+ * <ul>
+ *   <li>Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-compressed Binary,
+ *       along with the per-document ordinals written using one of the numeric strategies above.
+ * </ul>
+ *
+ * <p>{@link DocValuesType#SORTED_SET SORTED_SET}:
+ *
+ * <ul>
+ *   <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
+ *   <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Binary, an ordinal
+ *       list and per-document index into this list are written using the numeric strategies above.
+ * </ul>
+ *
+ * <p>{@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
+ *
+ * <ul>
+ *   <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
+ *   <li>SortedNumeric: a value list and per-document index into this list are written using the
+ *       numeric strategies above.
+ * </ul>
+ *
+ * <p>Files:
+ *
+ * <ol>
+ *   <li><code>.dvd</code>: DocValues data
+ *   <li><code>.dvm</code>: DocValues metadata
+ * </ol>
+ *
+ * @lucene.experimental
+ */
+public final class Lucene70DocValuesFormat extends DocValuesFormat {
+
+  /** Sole Constructor */
+  public Lucene70DocValuesFormat() {
+    super("Lucene70");
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene70DocValuesConsumer(
+        state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new Lucene70DocValuesProducer(
+        state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+
+  static final String DATA_CODEC = "Lucene70DocValuesData";
+  static final String DATA_EXTENSION = "dvd";
+  static final String META_CODEC = "Lucene70DocValuesMetadata";
+  static final String META_EXTENSION = "dvm";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+
+  // indicates docvalues type
+  static final byte NUMERIC = 0;
+  static final byte BINARY = 1;
+  static final byte SORTED = 2;
+  static final byte SORTED_SET = 3;
+  static final byte SORTED_NUMERIC = 4;
+
+  static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
+
+  static final int NUMERIC_BLOCK_SHIFT = 14;
+  static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
+
+  static final int TERMS_DICT_BLOCK_SHIFT = 4;
+  static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
+  static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;
+
+  static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10;
+  static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
+  static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1;
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesProducer.java
new file mode 100644
index 0000000..2e2f20d
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70DocValuesProducer.java
@@ -0,0 +1,1478 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.BaseTermsEnum;
+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.ImpactsEnum;
+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.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+import org.apache.lucene.util.packed.DirectReader;
+
+/** 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, SortedNumericEntry> sortedNumerics = new HashMap<>();
+  private long ramBytesUsed;
+  private final IndexInput data;
+  private final int maxDoc;
+
+  /** 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();
+    ramBytesUsed = RamUsageEstimator.shallowSizeOfInstance(getClass());
+
+    int version = -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);
+        readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
+    }
+
+    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 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) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      byte type = meta.readByte();
+      if (type == Lucene70DocValuesFormat.NUMERIC) {
+        numerics.put(info.name, readNumeric(meta));
+      } else if (type == Lucene70DocValuesFormat.BINARY) {
+        binaries.put(info.name, readBinary(meta));
+      } else if (type == Lucene70DocValuesFormat.SORTED) {
+        sorted.put(info.name, readSorted(meta));
+      } else if (type == Lucene70DocValuesFormat.SORTED_SET) {
+        sortedSets.put(info.name, readSortedSet(meta));
+      } else if (type == Lucene70DocValuesFormat.SORTED_NUMERIC) {
+        sortedNumerics.put(info.name, readSortedNumeric(meta));
+      } else {
+        throw new CorruptIndexException("invalid type: " + type, meta);
+      }
+    }
+  }
+
+  private NumericEntry readNumeric(ChecksumIndexInput meta) throws IOException {
+    NumericEntry entry = new NumericEntry();
+    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 > 256) {
+      throw new CorruptIndexException("invalid table size: " + tableSize, 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();
+      }
+    }
+    if (tableSize < -1) {
+      entry.blockShift = -2 - tableSize;
+    } else {
+      entry.blockShift = -1;
+    }
+    entry.bitsPerValue = meta.readByte();
+    entry.minValue = meta.readLong();
+    entry.gcd = meta.readLong();
+    entry.valuesOffset = meta.readLong();
+    entry.valuesLength = meta.readLong();
+  }
+
+  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();
+    }
+    return entry;
+  }
+
+  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("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;
+  }
+
+  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 void close() throws IOException {
+    data.close();
+  }
+
+  private static class NumericEntry {
+    long[] table;
+    int blockShift;
+    byte bitsPerValue;
+    long docsWithFieldOffset;
+    long docsWithFieldLength;
+    long numValues;
+    long minValue;
+    long gcd;
+    long valuesOffset;
+    long valuesLength;
+  }
+
+  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;
+  }
+
+  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;
+  }
+
+  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;
+  }
+
+  private static class SortedNumericEntry extends NumericEntry {
+    int numDocsWithField;
+    DirectMonotonicReader.Meta addressesMeta;
+    long addressesOffset;
+    long addressesLength;
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed;
+  }
+
+  @Override
+  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+    NumericEntry entry = numerics.get(field.name);
+    return getNumeric(entry);
+  }
+
+  private abstract static class DenseNumericDocValues extends NumericDocValues {
+
+    final int maxDoc;
+    int doc = -1;
+
+    DenseNumericDocValues(int maxDoc) {
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(doc + 1);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc) {
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = target;
+    }
+
+    @Override
+    public boolean advanceExact(int target) {
+      doc = target;
+      return true;
+    }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+  }
+
+  private abstract static class SparseNumericDocValues extends NumericDocValues {
+
+    final IndexedDISI disi;
+
+    SparseNumericDocValues(IndexedDISI disi) {
+      this.disi = disi;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return disi.advance(target);
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      return disi.advanceExact(target);
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return disi.nextDoc();
+    }
+
+    @Override
+    public int docID() {
+      return disi.docID();
+    }
+
+    @Override
+    public long cost() {
+      return disi.cost();
+    }
+  }
+
+  private NumericDocValues getNumeric(NumericEntry entry) throws IOException {
+    if (entry.docsWithFieldOffset == -2) {
+      // empty
+      return DocValues.emptyNumeric();
+    } else if (entry.docsWithFieldOffset == -1) {
+      // dense
+      if (entry.bitsPerValue == 0) {
+        return new DenseNumericDocValues(maxDoc) {
+          @Override
+          public long longValue() throws IOException {
+            return entry.minValue;
+          }
+        };
+      } else {
+        final RandomAccessInput slice =
+            data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
+        if (entry.blockShift >= 0) {
+          // dense but split into blocks of different bits per value
+          final int shift = entry.blockShift;
+          final long mul = entry.gcd;
+          final int mask = (1 << shift) - 1;
+          return new DenseNumericDocValues(maxDoc) {
+            int block = -1;
+            long delta;
+            long offset;
+            long blockEndOffset;
+            LongValues values;
+
+            @Override
+            public long longValue() throws IOException {
+              final int block = doc >>> shift;
+              if (this.block != block) {
+                int bitsPerValue;
+                do {
+                  offset = blockEndOffset;
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                  this.block++;
+                } while (this.block != block);
+                values =
+                    bitsPerValue == 0
+                        ? LongValues.ZEROES
+                        : DirectReader.getInstance(slice, bitsPerValue, offset);
+              }
+              return mul * values.get(doc & mask) + delta;
+            }
+          };
+        } else {
+          final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+          if (entry.table != null) {
+            final long[] table = entry.table;
+            return new DenseNumericDocValues(maxDoc) {
+              @Override
+              public long longValue() throws IOException {
+                return table[(int) values.get(doc)];
+              }
+            };
+          } else {
+            final long mul = entry.gcd;
+            final long delta = entry.minValue;
+            return new DenseNumericDocValues(maxDoc) {
+              @Override
+              public long longValue() throws IOException {
+                return mul * values.get(doc) + delta;
+              }
+            };
+          }
+        }
+      }
+    } else {
+      // sparse
+      final IndexedDISI disi =
+          new IndexedDISI(
+              data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numValues);
+      if (entry.bitsPerValue == 0) {
+        return new SparseNumericDocValues(disi) {
+          @Override
+          public long longValue() throws IOException {
+            return entry.minValue;
+          }
+        };
+      } else {
+        final RandomAccessInput slice =
+            data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
+        if (entry.blockShift >= 0) {
+          // sparse and split into blocks of different bits per value
+          final int shift = entry.blockShift;
+          final long mul = entry.gcd;
+          final int mask = (1 << shift) - 1;
+          return new SparseNumericDocValues(disi) {
+            int block = -1;
+            long delta;
+            long offset;
+            long blockEndOffset;
+            LongValues values;
+
+            @Override
+            public long longValue() throws IOException {
+              final int index = disi.index();
+              final int block = index >>> shift;
+              if (this.block != block) {
+                int bitsPerValue;
+                do {
+                  offset = blockEndOffset;
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                  this.block++;
+                } while (this.block != block);
+                values =
+                    bitsPerValue == 0
+                        ? LongValues.ZEROES
+                        : DirectReader.getInstance(slice, bitsPerValue, offset);
+              }
+              return mul * values.get(index & mask) + delta;
+            }
+          };
+        } else {
+          final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+          if (entry.table != null) {
+            final long[] table = entry.table;
+            return new SparseNumericDocValues(disi) {
+              @Override
+              public long longValue() throws IOException {
+                return table[(int) values.get(disi.index())];
+              }
+            };
+          } else {
+            final long mul = entry.gcd;
+            final long delta = entry.minValue;
+            return new SparseNumericDocValues(disi) {
+              @Override
+              public long longValue() throws IOException {
+                return mul * values.get(disi.index()) + delta;
+              }
+            };
+          }
+        }
+      }
+    }
+  }
+
+  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);
+      if (entry.blockShift >= 0) {
+        final int shift = entry.blockShift;
+        final long mul = entry.gcd;
+        final long mask = (1L << shift) - 1;
+        return new LongValues() {
+          long block = -1;
+          long delta;
+          long offset;
+          long blockEndOffset;
+          LongValues values;
+
+          public long get(long index) {
+            final long block = index >>> shift;
+            if (this.block != block) {
+              assert block > this.block
+                  : "Reading backwards is illegal: " + this.block + " < " + block;
+              int bitsPerValue;
+              do {
+                offset = blockEndOffset;
+                try {
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                } catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+                this.block++;
+              } while (this.block != block);
+              values =
+                  bitsPerValue == 0
+                      ? LongValues.ZEROES
+                      : DirectReader.getInstance(slice, bitsPerValue, offset);
+            }
+            return mul * values.get(index & mask) + delta;
+          }
+        };
+      } else {
+        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+        if (entry.table != null) {
+          final long[] table = entry.table;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return table[(int) values.get(index)];
+            }
+          };
+        } else if (entry.gcd != 1) {
+          final long gcd = entry.gcd;
+          final long minValue = entry.minValue;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return values.get(index) * gcd + minValue;
+            }
+          };
+        } else if (entry.minValue != 0) {
+          final long minValue = entry.minValue;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return values.get(index) + minValue;
+            }
+          };
+        } else {
+          return values;
+        }
+      }
+    }
+  }
+
+  private abstract static class DenseBinaryDocValues extends BinaryDocValues {
+
+    final int maxDoc;
+    int doc = -1;
+
+    DenseBinaryDocValues(int maxDoc) {
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(doc + 1);
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc) {
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = target;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      doc = target;
+      return true;
+    }
+  }
+
+  private abstract static class SparseBinaryDocValues extends BinaryDocValues {
+
+    final IndexedDISI disi;
+
+    SparseBinaryDocValues(IndexedDISI disi) {
+      this.disi = disi;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return disi.nextDoc();
+    }
+
+    @Override
+    public int docID() {
+      return disi.docID();
+    }
+
+    @Override
+    public long cost() {
+      return disi.cost();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return disi.advance(target);
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      return disi.advanceExact(target);
+    }
+  }
+
+  @Override
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryEntry entry = binaries.get(field.name);
+    if (entry.docsWithFieldOffset == -2) {
+      return DocValues.emptyBinary();
+    }
+
+    final IndexInput bytesSlice = data.slice("fixed-binary", entry.dataOffset, entry.dataLength);
+
+    if (entry.docsWithFieldOffset == -1) {
+      // dense
+      if (entry.minLength == entry.maxLength) {
+        // fixed length
+        final int length = entry.maxLength;
+        return new DenseBinaryDocValues(maxDoc) {
+          final BytesRef bytes = new BytesRef(new byte[length], 0, length);
+
+          @Override
+          public BytesRef binaryValue() throws IOException {
+            bytesSlice.seek((long) doc * length);
+            bytesSlice.readBytes(bytes.bytes, 0, length);
+            return bytes;
+          }
+        };
+      } else {
+        // variable length
+        final RandomAccessInput addressesData =
+            this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
+        final LongValues addresses =
+            DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
+        return new DenseBinaryDocValues(maxDoc) {
+          final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength);
+
+          @Override
+          public BytesRef binaryValue() throws IOException {
+            long startOffset = addresses.get(doc);
+            bytes.length = (int) (addresses.get(doc + 1L) - startOffset);
+            bytesSlice.seek(startOffset);
+            bytesSlice.readBytes(bytes.bytes, 0, bytes.length);
+            return bytes;
+          }
+        };
+      }
+    } else {
+      // sparse
+      final IndexedDISI disi =
+          new IndexedDISI(
+              data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
+      if (entry.minLength == entry.maxLength) {
+        // fixed length
+        final int length = entry.maxLength;
+        return new SparseBinaryDocValues(disi) {
+          final BytesRef bytes = new BytesRef(new byte[length], 0, length);
+
+          @Override
+          public BytesRef binaryValue() throws IOException {
+            bytesSlice.seek((long) disi.index() * length);
+            bytesSlice.readBytes(bytes.bytes, 0, length);
+            return bytes;
+          }
+        };
+      } else {
+        // variable length
+        final RandomAccessInput addressesData =
+            this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
+        final LongValues addresses =
+            DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
+        return new SparseBinaryDocValues(disi) {
+          final BytesRef bytes = new BytesRef(new byte[entry.maxLength], 0, entry.maxLength);
+
+          @Override
+          public BytesRef binaryValue() throws IOException {
+            final int index = disi.index();
+            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;
+          }
+        };
+      }
+    }
+  }
+
+  @Override
+  public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    SortedEntry entry = sorted.get(field.name);
+    return getSorted(entry);
+  }
+
+  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 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 advance(doc + 1);
+        }
+
+        @Override
+        public int docID() {
+          return doc;
+        }
+
+        @Override
+        public long cost() {
+          return maxDoc;
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          if (target >= maxDoc) {
+            return doc = NO_MORE_DOCS;
+          }
+          return doc = target;
+        }
+
+        @Override
+        public boolean advanceExact(int target) {
+          doc = target;
+          return true;
+        }
+
+        @Override
+        public int ordValue() {
+          return (int) ords.get(doc);
+        }
+      };
+    } else {
+      // sparse
+      final IndexedDISI disi =
+          new IndexedDISI(
+              data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
+      return new BaseSortedDocValues(entry, data) {
+
+        @Override
+        public int nextDoc() throws IOException {
+          return disi.nextDoc();
+        }
+
+        @Override
+        public int docID() {
+          return disi.docID();
+        }
+
+        @Override
+        public long cost() {
+          return disi.cost();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          return disi.advance(target);
+        }
+
+        @Override
+        public boolean advanceExact(int target) throws IOException {
+          return disi.advanceExact(target);
+        }
+
+        @Override
+        public int ordValue() {
+          return (int) ords.get(disi.index());
+        }
+      };
+    }
+  }
+
+  private abstract static class BaseSortedDocValues extends SortedDocValues {
+
+    final SortedEntry entry;
+    final IndexInput data;
+    final TermsEnum termsEnum;
+
+    BaseSortedDocValues(SortedEntry entry, IndexInput data) throws IOException {
+      this.entry = entry;
+      this.data = data;
+      this.termsEnum = termsEnum();
+    }
+
+    @Override
+    public int getValueCount() {
+      return Math.toIntExact(entry.termsDictSize);
+    }
+
+    @Override
+    public BytesRef lookupOrd(int ord) throws IOException {
+      termsEnum.seekExact(ord);
+      return termsEnum.term();
+    }
+
+    @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 abstract static class BaseSortedSetDocValues extends SortedSetDocValues {
+
+    final SortedSetEntry entry;
+    final IndexInput data;
+    final TermsEnum termsEnum;
+
+    BaseSortedSetDocValues(SortedSetEntry entry, IndexInput data) throws IOException {
+      this.entry = entry;
+      this.data = data;
+      this.termsEnum = termsEnum();
+    }
+
+    @Override
+    public long getValueCount() {
+      return entry.termsDictSize;
+    }
+
+    @Override
+    public BytesRef lookupOrd(long ord) throws IOException {
+      termsEnum.seekExact(ord);
+      return termsEnum.term();
+    }
+
+    @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 TermsEnum termsEnum() throws IOException {
+      return new TermsDict(entry, data);
+    }
+  }
+
+  private static class TermsDict extends BaseTermsEnum {
+
+    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 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 {
+          hi = mid - 1;
+        }
+      }
+
+      assert hi < 0 || getTermFromIndex(hi).compareTo(text) <= 0;
+      assert hi == ((entry.termsDictSize - 1) >>> entry.termsDictIndexShift)
+          || getTermFromIndex(hi + 1).compareTo(text) > 0;
+
+      return hi;
+    }
+
+    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;
+    }
+
+    private long seekBlock(BytesRef text) throws IOException {
+      long index = seekTermsIndex(text);
+      if (index == -1L) {
+        return -1L;
+      }
+
+      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 {
+          blockHi = blockMid - 1;
+        }
+      }
+
+      assert blockHi < 0 || getFirstTermFromBlock(blockHi).compareTo(text) <= 0;
+      assert blockHi == ((entry.termsDictSize - 1) >>> entry.termsDictBlockShift)
+          || getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0;
+
+      return blockHi;
+    }
+
+    @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 BytesRef term() throws IOException {
+      return term;
+    }
+
+    @Override
+    public long ord() throws IOException {
+      return ord;
+    }
+
+    @Override
+    public long totalTermFreq() throws IOException {
+      return -1L;
+    }
+
+    @Override
+    public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ImpactsEnum impacts(int flags) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int docFreq() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+    SortedNumericEntry entry = sortedNumerics.get(field.name);
+    if (entry.numValues == entry.numDocsWithField) {
+      return DocValues.singleton(getNumeric(entry));
+    }
+
+    final RandomAccessInput addressesInput =
+        data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
+    final LongValues addresses =
+        DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput);
+
+    final LongValues values = getNumericValues(entry);
+
+    if (entry.docsWithFieldOffset == -1) {
+      // dense
+      return new SortedNumericDocValues() {
+
+        int doc = -1;
+        long start, end;
+        int count;
+
+        @Override
+        public int nextDoc() throws IOException {
+          return advance(doc + 1);
+        }
+
+        @Override
+        public int docID() {
+          return doc;
+        }
+
+        @Override
+        public long cost() {
+          return maxDoc;
+        }
+
+        @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);
+          count = (int) (end - start);
+          return doc = target;
+        }
+
+        @Override
+        public boolean advanceExact(int target) throws IOException {
+          start = addresses.get(target);
+          end = addresses.get(target + 1L);
+          count = (int) (end - start);
+          doc = target;
+          return true;
+        }
+
+        @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() {
+
+        boolean set;
+        long start, end;
+        int count;
+
+        @Override
+        public int nextDoc() throws IOException {
+          set = false;
+          return disi.nextDoc();
+        }
+
+        @Override
+        public int docID() {
+          return disi.docID();
+        }
+
+        @Override
+        public long cost() {
+          return disi.cost();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          set = false;
+          return disi.advance(target);
+        }
+
+        @Override
+        public boolean advanceExact(int target) throws IOException {
+          set = false;
+          return disi.advanceExact(target);
+        }
+
+        @Override
+        public long nextValue() throws IOException {
+          set();
+          return values.get(start++);
+        }
+
+        @Override
+        public int docValueCount() {
+          set();
+          return count;
+        }
+
+        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 SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    SortedSetEntry entry = sortedSets.get(field.name);
+    if (entry.singleValueEntry != null) {
+      return DocValues.singleton(getSorted(entry.singleValueEntry));
+    }
+
+    final RandomAccessInput slice = data.randomAccessSlice(entry.ordsOffset, entry.ordsLength);
+    final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue);
+
+    final RandomAccessInput addressesInput =
+        data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
+    final LongValues addresses =
+        DirectMonotonicReader.getInstance(entry.addressesMeta, addressesInput);
+
+    if (entry.docsWithFieldOffset == -1) {
+      // dense
+      return new BaseSortedSetDocValues(entry, data) {
+
+        int doc = -1;
+        long start;
+        long end;
+
+        @Override
+        public int nextDoc() throws IOException {
+          return advance(doc + 1);
+        }
+
+        @Override
+        public int docID() {
+          return doc;
+        }
+
+        @Override
+        public long cost() {
+          return maxDoc;
+        }
+
+        @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;
+        }
+
+        @Override
+        public boolean advanceExact(int target) throws IOException {
+          start = addresses.get(target);
+          end = addresses.get(target + 1L);
+          doc = target;
+          return true;
+        }
+
+        @Override
+        public long nextOrd() throws IOException {
+          if (start == end) {
+            return NO_MORE_ORDS;
+          }
+          return ords.get(start++);
+        }
+      };
+    } else {
+      // sparse
+      final IndexedDISI disi =
+          new IndexedDISI(
+              data, entry.docsWithFieldOffset, entry.docsWithFieldLength, entry.numDocsWithField);
+      return new BaseSortedSetDocValues(entry, data) {
+
+        boolean set;
+        long start;
+        long end = 0;
+
+        @Override
+        public int nextDoc() throws IOException {
+          set = false;
+          return disi.nextDoc();
+        }
+
+        @Override
+        public int docID() {
+          return disi.docID();
+        }
+
+        @Override
+        public long cost() {
+          return disi.cost();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          set = false;
+          return disi.advance(target);
+        }
+
+        @Override
+        public boolean advanceExact(int target) throws IOException {
+          set = false;
+          return disi.advanceExact(target);
+        }
+
+        @Override
+        public long nextOrd() throws IOException {
+          if (set == false) {
+            final int index = disi.index();
+            final long start = addresses.get(index);
+            this.start = start + 1;
+            end = addresses.get(index + 1L);
+            set = true;
+            return ords.get(start);
+          } else if (start == end) {
+            return NO_MORE_ORDS;
+          } else {
+            return ords.get(start++);
+          }
+        }
+      };
+    }
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(data);
+  }
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsConsumer.java
new file mode 100644
index 0000000..2314183
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsConsumer.java
@@ -0,0 +1,169 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+
+/** Writer for {@link Lucene70NormsFormat} */
+final class Lucene70NormsConsumer extends NormsConsumer {
+  IndexOutput data, meta;
+  final int maxDoc;
+
+  Lucene70NormsConsumer(
+      SegmentWriteState state,
+      String dataCodec,
+      String dataExtension,
+      String metaCodec,
+      String metaExtension)
+      throws IOException {
+    boolean success = false;
+    try {
+      String dataName =
+          IndexFileNames.segmentFileName(
+              state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeIndexHeader(
+          data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      String metaName =
+          IndexFileNames.segmentFileName(
+              state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeIndexHeader(
+          meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      maxDoc = state.segmentInfo.maxDoc();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeInt(-1); // write EOF marker
+        CodecUtil.writeFooter(meta); // write checksum
+      }
+      if (data != null) {
+        CodecUtil.writeFooter(data); // write checksum
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+      meta = data = null;
+    }
+  }
+
+  @Override
+  public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
+    NumericDocValues values = normsProducer.getNorms(field);
+    int numDocsWithValue = 0;
+    long min = Long.MAX_VALUE;
+    long max = Long.MIN_VALUE;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      numDocsWithValue++;
+      long v = values.longValue();
+      min = Math.min(min, v);
+      max = Math.max(max, v);
+    }
+    assert numDocsWithValue <= maxDoc;
+
+    meta.writeInt(field.number);
+
+    if (numDocsWithValue == 0) {
+      meta.writeLong(-2);
+      meta.writeLong(0L);
+    } else if (numDocsWithValue == maxDoc) {
+      meta.writeLong(-1);
+      meta.writeLong(0L);
+    } else {
+      long offset = data.getFilePointer();
+      meta.writeLong(offset);
+      values = normsProducer.getNorms(field);
+      IndexedDISI.writeBitSet(values, data);
+      meta.writeLong(data.getFilePointer() - offset);
+    }
+
+    meta.writeInt(numDocsWithValue);
+    int numBytesPerValue = numBytesPerValue(min, max);
+
+    meta.writeByte((byte) numBytesPerValue);
+    if (numBytesPerValue == 0) {
+      meta.writeLong(min);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      values = normsProducer.getNorms(field);
+      writeValues(values, numBytesPerValue, data);
+    }
+  }
+
+  private int numBytesPerValue(long min, long max) {
+    if (min >= max) {
+      return 0;
+    } else if (min >= Byte.MIN_VALUE && max <= Byte.MAX_VALUE) {
+      return 1;
+    } else if (min >= Short.MIN_VALUE && max <= Short.MAX_VALUE) {
+      return 2;
+    } else if (min >= Integer.MIN_VALUE && max <= Integer.MAX_VALUE) {
+      return 4;
+    } else {
+      return 8;
+    }
+  }
+
+  private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out)
+      throws IOException, AssertionError {
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      long value = values.longValue();
+      switch (numBytesPerValue) {
+        case 1:
+          out.writeByte((byte) value);
+          break;
+        case 2:
+          out.writeShort((short) value);
+          break;
+        case 4:
+          out.writeInt((int) value);
+          break;
+        case 8:
+          out.writeLong(value);
+          break;
+        default:
+          throw new AssertionError();
+      }
+    }
+  }
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsFormat.java
new file mode 100644
index 0000000..354574d
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsFormat.java
@@ -0,0 +1,106 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
+
+/**
+ * Lucene 7.0 Score normalization format.
+ *
+ * <p>Encodes normalization values by encoding each value with the minimum number of bytes needed to
+ * represent the range (which can be zero).
+ *
+ * <p>Files:
+ *
+ * <ol>
+ *   <li><code>.nvd</code>: Norms data
+ *   <li><code>.nvm</code>: Norms metadata
+ * </ol>
+ *
+ * <ol>
+ *   <li><a id="nvm"></a>
+ *       <p>The Norms metadata or .nvm file.
+ *       <p>For each norms field, this stores metadata, such as the offset into the Norms data
+ *       (.nvd)
+ *       <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer
+ *       <ul>
+ *         <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
+ *         <li>Entry --&gt; FieldNumber, DocsWithFieldAddress, DocsWithFieldLength,
+ *             NumDocsWithField, BytesPerNorm, NormsAddress
+ *         <li>FieldNumber --&gt; {@link DataOutput#writeInt Int32}
+ *         <li>DocsWithFieldAddress --&gt; {@link DataOutput#writeLong Int64}
+ *         <li>DocsWithFieldLength --&gt; {@link DataOutput#writeLong Int64}
+ *         <li>NumDocsWithField --&gt; {@link DataOutput#writeInt Int32}
+ *         <li>BytesPerNorm --&gt; {@link DataOutput#writeByte byte}
+ *         <li>NormsAddress --&gt; {@link DataOutput#writeLong Int64}
+ *         <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
+ *       </ul>
+ *       <p>FieldNumber of -1 indicates the end of metadata.
+ *       <p>NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the
+ *       singleton value when BytesPerValue = 0. If BytesPerValue is different from 0 then there are
+ *       NumDocsWithField values to read at that offset.
+ *       <p>DocsWithFieldAddress is the pointer to the start of the bit set containing documents
+ *       that have a norm in the norms data (.nvd), or -2 if no documents have a norm value, or -1
+ *       if all documents have a norm value.
+ *       <p>DocsWithFieldLength is the number of bytes used to encode the set of documents that have
+ *       a norm.
+ *   <li><a id="nvd"></a>
+ *       <p>The Norms data or .nvd file.
+ *       <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)
+ *       <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer
+ *       <ul>
+ *         <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
+ *         <li>DocsWithFieldData --&gt; Bit set of MaxDoc bits
+ *         <li>NormsData --&gt; {@link DataOutput#writeByte(byte) byte}<sup>NumDocsWithField *
+ *             BytesPerValue</sup>
+ *         <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
+ *       </ul>
+ * </ol>
+ *
+ * @lucene.experimental
+ */
+public class Lucene70NormsFormat extends NormsFormat {
+
+  /** Sole Constructor */
+  public Lucene70NormsFormat() {}
+
+  @Override
+  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene70NormsConsumer(
+        state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+
+  @Override
+  public NormsProducer normsProducer(SegmentReadState state) throws IOException {
+    return new Lucene70NormsProducer(
+        state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+
+  private static final String DATA_CODEC = "Lucene70NormsData";
+  private static final String DATA_EXTENSION = "nvd";
+  private static final String METADATA_CODEC = "Lucene70NormsMetadata";
+  private static final String METADATA_EXTENSION = "nvm";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+}
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsProducer.java
new file mode 100644
index 0000000..1446199
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene70/Lucene70NormsProducer.java
@@ -0,0 +1,464 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
+import static org.apache.lucene.backward_codecs.lucene70.Lucene70NormsFormat.VERSION_START;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.NormsProducer;
+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.SegmentReadState;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.IOUtils;
+
+/** Reader for {@link Lucene70NormsFormat} */
+final class Lucene70NormsProducer extends NormsProducer implements Cloneable {
+  // metadata maps (just file pointers and minimal stuff)
+  private final Map<Integer, NormsEntry> norms = new HashMap<>();
+  private final int maxDoc;
+  private IndexInput data;
+  private boolean merging;
+  private Map<Integer, IndexInput> disiInputs;
+  private Map<Integer, RandomAccessInput> dataInputs;
+
+  Lucene70NormsProducer(
+      SegmentReadState state,
+      String dataCodec,
+      String dataExtension,
+      String metaCodec,
+      String metaExtension)
+      throws IOException {
+    maxDoc = state.segmentInfo.maxDoc();
+    String metaName =
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+    int version = -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,
+                VERSION_START,
+                VERSION_CURRENT,
+                state.segmentInfo.getId(),
+                state.segmentSuffix);
+        readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
+    }
+
+    String dataName =
+        IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+    data = state.directory.openInput(dataName, state.context);
+    boolean success = false;
+    try {
+      final int version2 =
+          CodecUtil.checkIndexHeader(
+              data,
+              dataCodec,
+              VERSION_START,
+              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);
+      }
+    }
+  }
+
+  @Override
+  public NormsProducer getMergeInstance() {
+    Lucene70NormsProducer clone;
+    try {
+      clone = (Lucene70NormsProducer) super.clone();
+    } catch (CloneNotSupportedException e) {
+      // cannot happen
+      throw new RuntimeException(e);
+    }
+    clone.data = data.clone();
+    clone.dataInputs = new HashMap<>();
+    clone.disiInputs = new HashMap<>();
+    clone.merging = true;
+    return clone;
+  }
+
+  static class NormsEntry {
+    byte bytesPerNorm;
+    long docsWithFieldOffset;
+    long docsWithFieldLength;
+    int numDocsWithField;
+    long normsOffset;
+  }
+
+  abstract static class DenseNormsIterator extends NumericDocValues {
+
+    final int maxDoc;
+    int doc = -1;
+
+    DenseNormsIterator(int maxDoc) {
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(doc + 1);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target >= maxDoc) {
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = target;
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      this.doc = target;
+      return true;
+    }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+  }
+
+  abstract static class SparseNormsIterator extends NumericDocValues {
+
+    final IndexedDISI disi;
+
+    SparseNormsIterator(IndexedDISI disi) {
+      this.disi = disi;
+    }
+
+    @Override
+    public int docID() {
+      return disi.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return disi.nextDoc();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return disi.advance(target);
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      return disi.advanceExact(target);
+    }
+
+    @Override
+    public long cost() {
+      return disi.cost();
+    }
+  }
+
+  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      } else if (!info.hasNorms()) {
+        throw new CorruptIndexException("Invalid field: " + info.name, meta);
+      }
+      NormsEntry entry = new NormsEntry();
+      entry.docsWithFieldOffset = meta.readLong();
+      entry.docsWithFieldLength = meta.readLong();
+      entry.numDocsWithField = meta.readInt();
+      entry.bytesPerNorm = meta.readByte();
+      switch (entry.bytesPerNorm) {
+        case 0:
+        case 1:
+        case 2:
+        case 4:
+        case 8:
+          break;
+        default:
+          throw new CorruptIndexException(
+              "Invalid bytesPerValue: " + entry.bytesPerNorm + ", field: " + info.name, meta);
+      }
+      entry.normsOffset = meta.readLong();
+      norms.put(info.number, entry);
+    }
+  }
+
+  private RandomAccessInput getDataInput(FieldInfo field, NormsEntry entry) throws IOException {
+    RandomAccessInput slice = null;
+    if (merging) {
+      slice = dataInputs.get(field.number);
+    }
+    if (slice == null) {
+      slice =
+          data.randomAccessSlice(
+              entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
+      if (merging) {
+        dataInputs.put(field.number, slice);
+      }
+    }
+    return slice;
+  }
+
+  private IndexInput getDisiInput(FieldInfo field, NormsEntry entry) throws IOException {
+    if (merging == false) {
+      return data.slice("docs", entry.docsWithFieldOffset, entry.docsWithFieldLength);
+    }
+
+    IndexInput in = disiInputs.get(field.number);
+    if (in == null) {
+      in = data.slice("docs", entry.docsWithFieldOffset, entry.docsWithFieldLength);
+      disiInputs.put(field.number, in);
+    }
+
+    final IndexInput inF = in; // same as in but final
+
+    // Wrap so that reads can be interleaved from the same thread if two
+    // norms instances are pulled and consumed in parallel. Merging usually
+    // doesn't need this feature but CheckIndex might, plus we need merge
+    // instances to behave well and not be trappy.
+    return new IndexInput("docs") {
+
+      long offset = 0;
+
+      @Override
+      public void readBytes(byte[] b, int off, int len) throws IOException {
+        inF.seek(offset);
+        offset += len;
+        inF.readBytes(b, off, len);
+      }
+
+      @Override
+      public byte readByte() throws IOException {
+        throw new UnsupportedOperationException("Unused by IndexedDISI");
+      }
+
+      @Override
+      public IndexInput slice(String sliceDescription, long offset, long length)
+          throws IOException {
+        throw new UnsupportedOperationException("Unused by IndexedDISI");
+      }
+
+      @Override
+      public short readShort() throws IOException {
+        inF.seek(offset);
+        offset += Short.BYTES;
+        return inF.readShort();
+      }
+
+      @Override
+      public long readLong() throws IOException {
+        inF.seek(offset);
+        offset += Long.BYTES;
+        return inF.readLong();
+      }
+
+      @Override
+      public void seek(long pos) throws IOException {
+        offset = pos;
+      }
+
+      @Override
+      public long length() {
+        throw new UnsupportedOperationException("Unused by IndexedDISI");
+      }
+
+      @Override
+      public long getFilePointer() {
+        return offset;
+      }
+
+      @Override
+      public void close() throws IOException {
+        throw new UnsupportedOperationException("Unused by IndexedDISI");
+      }
+    };
+  }
+
+  private IndexInput getDisiInput2(FieldInfo field, NormsEntry entry) throws IOException {
+    IndexInput slice = null;
+    if (merging) {
+      slice = disiInputs.get(field.number);
+    }
+    if (slice == null) {
+      slice = data.slice("docs", entry.docsWithFieldOffset, entry.docsWithFieldLength);
+      if (merging) {
+        disiInputs.put(field.number, slice);
+      }
+    }
+    return slice;
+  }
+
+  @Override
+  public NumericDocValues getNorms(FieldInfo field) throws IOException {
+    final NormsEntry entry = norms.get(field.number);
+    if (entry.docsWithFieldOffset == -2) {
+      // empty
+      return DocValues.emptyNumeric();
+    } else if (entry.docsWithFieldOffset == -1) {
+      // dense
+      if (entry.bytesPerNorm == 0) {
+        return new DenseNormsIterator(maxDoc) {
+          @Override
+          public long longValue() throws IOException {
+            return entry.normsOffset;
+          }
+        };
+      }
+      final RandomAccessInput slice = getDataInput(field, entry);
+      switch (entry.bytesPerNorm) {
+        case 1:
+          return new DenseNormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readByte(doc);
+            }
+          };
+        case 2:
+          return new DenseNormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readShort(((long) doc) << 1);
+            }
+          };
+        case 4:
+          return new DenseNormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readInt(((long) doc) << 2);
+            }
+          };
+        case 8:
+          return new DenseNormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readLong(((long) doc) << 3);
+            }
+          };
+        default:
+          // should not happen, we already validate bytesPerNorm in readFields
+          throw new AssertionError();
+      }
+    } else {
+      // sparse
+      final IndexInput disiInput = getDisiInput(field, entry);
+      final IndexedDISI disi = new IndexedDISI(disiInput, entry.numDocsWithField);
+      if (entry.bytesPerNorm == 0) {
+        return new SparseNormsIterator(disi) {
+          @Override
+          public long longValue() throws IOException {
+            return entry.normsOffset;
+          }
+        };
+      }
+      final RandomAccessInput slice =
+          data.randomAccessSlice(
+              entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
+      switch (entry.bytesPerNorm) {
+        case 1:
+          return new SparseNormsIterator(disi) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readByte(disi.index());
+            }
+          };
+        case 2:
+          return new SparseNormsIterator(disi) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readShort(((long) disi.index()) << 1);
+            }
+          };
+        case 4:
+          return new SparseNormsIterator(disi) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readInt(((long) disi.index()) << 2);
+            }
+          };
+        case 8:
+          return new SparseNormsIterator(disi) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readLong(((long) disi.index()) << 3);
+            }
+          };
+        default:
+          // should not happen, we already validate bytesPerNorm in readFields
+          throw new AssertionError();
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    data.close();
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 64L * norms.size(); // good enough
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(data);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
+  }
+}
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index c9a6322..5ff3ebf 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -17,3 +17,4 @@ org.apache.lucene.backward_codecs.lucene80.Lucene80Codec
 org.apache.lucene.backward_codecs.lucene84.Lucene84Codec
 org.apache.lucene.backward_codecs.lucene86.Lucene86Codec
 org.apache.lucene.backward_codecs.lucene87.Lucene87Codec
+org.apache.lucene.backward_codecs.lucene70.Lucene70Codec
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
index 4a812de..8f034f8 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
@@ -13,3 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
+org.apache.lucene.backward_codecs.lucene70.Lucene70DocValuesFormat
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWCodec.java
new file mode 100644
index 0000000..bb1465f
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/Lucene70RWCodec.java
@@ -0,0 +1,55 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import org.apache.lucene.backward_codecs.lucene50.Lucene50RWPostingsFormat;
+import org.apache.lucene.backward_codecs.lucene50.Lucene50RWStoredFieldsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/** RW impersonation of {@link Lucene70Codec}. */
+public final class Lucene70RWCodec extends Lucene70Codec {
+
+  private final PostingsFormat defaultPF = new Lucene50RWPostingsFormat();
+  private final PostingsFormat postingsFormat =
+      new PerFieldPostingsFormat() {
+        @Override
+        public PostingsFormat getPostingsFormatForField(String field) {
+          return defaultPF;
+        }
+      };
+
+  /** Sole constructor. */
+  public Lucene70RWCodec() {}
+
+  @Override
+  public SegmentInfoFormat segmentInfoFormat() {
+    return new Lucene70RWSegmentInfoFormat();
+  }
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return new Lucene50RWStoredFieldsFormat();
+  }
+
+  @Override
+  public PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestIndexedDISI.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestIndexedDISI.java
new file mode 100644
index 0000000..a271a27
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestIndexedDISI.java
@@ -0,0 +1,248 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestIndexedDISI extends LuceneTestCase {
+
+  public void testEmpty() throws IOException {
+    int maxDoc = TestUtil.nextInt(random(), 1, 100000);
+    FixedBitSet set = new FixedBitSet(maxDoc);
+    try (Directory dir = newDirectory()) {
+      doTest(set, dir);
+    }
+  }
+
+  public void testOneDoc() throws IOException {
+    int maxDoc = TestUtil.nextInt(random(), 1, 100000);
+    FixedBitSet set = new FixedBitSet(maxDoc);
+    set.set(random().nextInt(maxDoc));
+    try (Directory dir = newDirectory()) {
+      doTest(set, dir);
+    }
+  }
+
+  public void testTwoDocs() throws IOException {
+    int maxDoc = TestUtil.nextInt(random(), 1, 100000);
+    FixedBitSet set = new FixedBitSet(maxDoc);
+    set.set(random().nextInt(maxDoc));
+    set.set(random().nextInt(maxDoc));
+    try (Directory dir = newDirectory()) {
+      doTest(set, dir);
+    }
+  }
+
+  public void testAllDocs() throws IOException {
+    int maxDoc = TestUtil.nextInt(random(), 1, 100000);
+    FixedBitSet set = new FixedBitSet(maxDoc);
+    set.set(1, maxDoc);
+    try (Directory dir = newDirectory()) {
+      doTest(set, dir);
+    }
+  }
+
+  public void testHalfFull() throws IOException {
+    int maxDoc = TestUtil.nextInt(random(), 1, 100000);
+    FixedBitSet set = new FixedBitSet(maxDoc);
+    for (int i = random().nextInt(2); i < maxDoc; i += TestUtil.nextInt(random(), 1, 3)) {
+      set.set(i);
+    }
+    try (Directory dir = newDirectory()) {
+      doTest(set, dir);
+    }
+  }
+
+  public void testDocRange() throws IOException {
+    try (Directory dir = newDirectory()) {
+      for (int iter = 0; iter < 10; ++iter) {
+        int maxDoc = TestUtil.nextInt(random(), 1, 1000000);
+        FixedBitSet set = new FixedBitSet(maxDoc);
+        final int start = random().nextInt(maxDoc);
+        final int end = TestUtil.nextInt(random(), start + 1, maxDoc);
+        set.set(start, end);
+        doTest(set, dir);
+      }
+    }
+  }
+
+  public void testSparseDenseBoundary() throws IOException {
+    try (Directory dir = newDirectory()) {
+      FixedBitSet set = new FixedBitSet(200000);
+      int start = 65536 + random().nextInt(100);
+
+      // we set MAX_ARRAY_LENGTH bits so the encoding will be sparse
+      set.set(start, start + IndexedDISI.MAX_ARRAY_LENGTH);
+      long length;
+      try (IndexOutput out = dir.createOutput("sparse", IOContext.DEFAULT)) {
+        IndexedDISI.writeBitSet(new BitSetIterator(set, IndexedDISI.MAX_ARRAY_LENGTH), out);
+        length = out.getFilePointer();
+      }
+      try (IndexInput in = dir.openInput("sparse", IOContext.DEFAULT)) {
+        IndexedDISI disi = new IndexedDISI(in, 0L, length, IndexedDISI.MAX_ARRAY_LENGTH);
+        assertEquals(start, disi.nextDoc());
+        assertEquals(IndexedDISI.Method.SPARSE, disi.method);
+      }
+      doTest(set, dir);
+
+      // now we set one more bit so the encoding will be dense
+      set.set(start + IndexedDISI.MAX_ARRAY_LENGTH + random().nextInt(100));
+      try (IndexOutput out = dir.createOutput("bar", IOContext.DEFAULT)) {
+        IndexedDISI.writeBitSet(new BitSetIterator(set, IndexedDISI.MAX_ARRAY_LENGTH + 1), out);
+        length = out.getFilePointer();
+      }
+      try (IndexInput in = dir.openInput("bar", IOContext.DEFAULT)) {
+        IndexedDISI disi = new IndexedDISI(in, 0L, length, IndexedDISI.MAX_ARRAY_LENGTH + 1);
+        assertEquals(start, disi.nextDoc());
+        assertEquals(IndexedDISI.Method.DENSE, disi.method);
+      }
+      doTest(set, dir);
+    }
+  }
+
+  public void testOneDocMissing() throws IOException {
+    int maxDoc = TestUtil.nextInt(random(), 1, 1000000);
+    FixedBitSet set = new FixedBitSet(maxDoc);
+    set.set(0, maxDoc);
+    set.clear(random().nextInt(maxDoc));
+    try (Directory dir = newDirectory()) {
+      doTest(set, dir);
+    }
+  }
+
+  public void testFewMissingDocs() throws IOException {
+    try (Directory dir = newDirectory()) {
+      for (int iter = 0; iter < 100; ++iter) {
+        int maxDoc = TestUtil.nextInt(random(), 1, 100000);
+        FixedBitSet set = new FixedBitSet(maxDoc);
+        set.set(0, maxDoc);
+        final int numMissingDocs = TestUtil.nextInt(random(), 2, 1000);
+        for (int i = 0; i < numMissingDocs; ++i) {
+          set.clear(random().nextInt(maxDoc));
+        }
+        doTest(set, dir);
+      }
+    }
+  }
+
+  public void testRandom() throws IOException {
+    try (Directory dir = newDirectory()) {
+      for (int i = 0; i < 10; ++i) {
+        doTestRandom(dir);
+      }
+    }
+  }
+
+  private void doTestRandom(Directory dir) throws IOException {
+    List<Integer> docs = new ArrayList<>();
+    final int maxStep = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 2, 20));
+    final int numDocs =
+        TestUtil.nextInt(random(), 1, Math.min(100000, Integer.MAX_VALUE / maxStep));
+    for (int doc = -1, i = 0; i < numDocs; ++i) {
+      doc += TestUtil.nextInt(random(), 1, maxStep);
+      docs.add(doc);
+    }
+    final int maxDoc = docs.get(docs.size() - 1) + TestUtil.nextInt(random(), 1, 100);
+
+    FixedBitSet set = new FixedBitSet(maxDoc);
+    for (int doc : docs) {
+      set.set(doc);
+    }
+
+    doTest(set, dir);
+  }
+
+  private void doTest(FixedBitSet set, Directory dir) throws IOException {
+    final int cardinality = set.cardinality();
+    long length;
+    try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) {
+      IndexedDISI.writeBitSet(new BitSetIterator(set, cardinality), out);
+      length = out.getFilePointer();
+    }
+
+    try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
+      IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
+      BitSetIterator disi2 = new BitSetIterator(set, cardinality);
+      int i = 0;
+      for (int doc = disi2.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = disi2.nextDoc()) {
+        assertEquals(doc, disi.nextDoc());
+        assertEquals(i++, disi.index());
+      }
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, disi.nextDoc());
+    }
+
+    for (int step : new int[] {1, 10, 100, 1000, 10000, 100000}) {
+      try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
+        IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
+        BitSetIterator disi2 = new BitSetIterator(set, cardinality);
+        int index = -1;
+        while (true) {
+          int target = disi2.docID() + step;
+          int doc;
+          do {
+            doc = disi2.nextDoc();
+            index++;
+          } while (doc < target);
+          assertEquals(doc, disi.advance(target));
+          if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+            break;
+          }
+          assertEquals(index, disi.index());
+        }
+      }
+    }
+
+    for (int step : new int[] {10, 100, 1000, 10000, 100000}) {
+      try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
+        IndexedDISI disi = new IndexedDISI(in, 0L, length, cardinality);
+        BitSetIterator disi2 = new BitSetIterator(set, cardinality);
+        int index = -1;
+        for (int target = 0; target < set.length(); ) {
+          target += TestUtil.nextInt(random(), 0, step);
+          int doc = disi2.docID();
+          while (doc < target) {
+            doc = disi2.nextDoc();
+            index++;
+          }
+
+          boolean exists = disi.advanceExact(target);
+          assertEquals(doc == target, exists);
+          if (exists) {
+            assertEquals(index, disi.index());
+          } else if (random().nextBoolean()) {
+            assertEquals(doc, disi.nextDoc());
+            assertEquals(index, disi.index());
+            target = doc;
+          }
+        }
+      }
+    }
+
+    dir.deleteFile("foo");
+  }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70DocValuesFormat.java
new file mode 100644
index 0000000..2f3fbb6
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70DocValuesFormat.java
@@ -0,0 +1,694 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.asserting.AssertingCodec;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.TestUtil;
+
+/** Tests Lucene70DocValuesFormat */
+public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene70DocValuesFormat());
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+
+  // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
+  // for now just keep them here, as we want to test this for this format.
+
+  @Slow
+  public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
+    }
+  }
+
+  @Nightly
+  public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
+    }
+  }
+
+  @Slow
+  public void testSortedVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedVsStoredFields(atLeast(300), 1d, 1, 32766);
+    }
+  }
+
+  @Nightly
+  public void testSortedVariableLengthManyVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1d, 1, 500);
+    }
+  }
+
+  @Slow
+  public void testTermsEnumFixedWidth() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(
+          TestUtil.nextInt(random(), 1025, 5121),
+          () -> TestUtil.randomSimpleString(random(), 10, 10));
+    }
+  }
+
+  @Slow
+  public void testTermsEnumVariableWidth() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(
+          TestUtil.nextInt(random(), 1025, 5121),
+          () -> TestUtil.randomSimpleString(random(), 1, 500));
+    }
+  }
+
+  @Nightly
+  public void testTermsEnumRandomMany() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(
+          TestUtil.nextInt(random(), 1025, 8121),
+          () -> TestUtil.randomSimpleString(random(), 1, 500));
+    }
+  }
+
+  public void testTermsEnumLongSharedPrefixes() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestTermsEnumRandom(
+          TestUtil.nextInt(random(), 1025, 5121),
+          () -> {
+            char[] chars = new char[random().nextInt(500)];
+            Arrays.fill(chars, 'a');
+            if (chars.length > 0) {
+              chars[random().nextInt(chars.length)] = 'b';
+            }
+            return new String(chars);
+          });
+    }
+  }
+
+  @Slow
+  public void testSparseDocValuesVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSparseDocValuesVsStoredFields();
+    }
+  }
+
+  private void doTestSparseDocValuesVsStoredFields() throws Exception {
+    final long[] values = new long[TestUtil.nextInt(random(), 1, 500)];
+    for (int i = 0; i < values.length; ++i) {
+      values[i] = random().nextLong();
+    }
+
+    Directory dir = newFSDirectory(createTempDir());
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMergeScheduler(new SerialMergeScheduler());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+
+    // sparse compression is only enabled if less than 1% of docs have a value
+    final int avgGap = 100;
+
+    final int numDocs = atLeast(200);
+    for (int i = random().nextInt(avgGap * 2); i >= 0; --i) {
+      writer.addDocument(new Document());
+    }
+    final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+
+      // single-valued
+      long docValue = values[random().nextInt(values.length)];
+      doc.add(new NumericDocValuesField("numeric", docValue));
+      doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue))));
+      doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue))));
+      doc.add(new StoredField("value", docValue));
+
+      // multi-valued
+      final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc);
+      for (int j = 0; j < numValues; ++j) {
+        docValue = values[random().nextInt(values.length)];
+        doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue));
+        doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue))));
+        doc.add(new StoredField("values", docValue));
+      }
+
+      writer.addDocument(doc);
+
+      // add a gap
+      for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) {
+        writer.addDocument(new Document());
+      }
+    }
+
+    if (random().nextBoolean()) {
+      writer.forceMerge(1);
+    }
+
+    final IndexReader indexReader = writer.getReader();
+    writer.close();
+
+    for (LeafReaderContext context : indexReader.leaves()) {
+      final LeafReader reader = context.reader();
+      final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric");
+
+      final SortedDocValues sorted = DocValues.getSorted(reader, "sorted");
+
+      final BinaryDocValues binary = DocValues.getBinary(reader, "binary");
+
+      final SortedNumericDocValues sortedNumeric =
+          DocValues.getSortedNumeric(reader, "sorted_numeric");
+
+      final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set");
+
+      for (int i = 0; i < reader.maxDoc(); ++i) {
+        final Document doc = reader.document(i);
+        final IndexableField valueField = doc.getField("value");
+        final Long value = valueField == null ? null : valueField.numericValue().longValue();
+
+        if (value == null) {
+          assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i);
+        } else {
+          assertEquals(i, numeric.nextDoc());
+          assertEquals(i, binary.nextDoc());
+          assertEquals(i, sorted.nextDoc());
+          assertEquals(value.longValue(), numeric.longValue());
+          assertTrue(sorted.ordValue() >= 0);
+          assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue()));
+          assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue());
+        }
+
+        final IndexableField[] valuesFields = doc.getFields("values");
+        if (valuesFields.length == 0) {
+          assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i);
+        } else {
+          final Set<Long> valueSet = new HashSet<>();
+          for (IndexableField sf : valuesFields) {
+            valueSet.add(sf.numericValue().longValue());
+          }
+
+          assertEquals(i, sortedNumeric.nextDoc());
+          assertEquals(valuesFields.length, sortedNumeric.docValueCount());
+          for (int j = 0; j < sortedNumeric.docValueCount(); ++j) {
+            assertTrue(valueSet.contains(sortedNumeric.nextValue()));
+          }
+          assertEquals(i, sortedSet.nextDoc());
+          int sortedSetCount = 0;
+          while (true) {
+            long ord = sortedSet.nextOrd();
+            if (ord == SortedSetDocValues.NO_MORE_ORDS) {
+              break;
+            }
+            assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
+            sortedSetCount++;
+          }
+          assertEquals(valueSet.size(), sortedSetCount);
+        }
+      }
+    }
+
+    indexReader.close();
+    dir.close();
+  }
+
+  // TODO: try to refactor this and some termsenum tests into the base class.
+  // to do this we need to fix the test class to get a DVF not a Codec so we can setup
+  // the postings format correctly.
+  private void doTestTermsEnumRandom(int numDocs, Supplier<String> valuesProducer)
+      throws Exception {
+    Directory dir = newFSDirectory(createTempDir());
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMergeScheduler(new SerialMergeScheduler());
+    // set to duel against a codec which has ordinals:
+    final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
+    final DocValuesFormat dv = new Lucene70DocValuesFormat();
+    conf.setCodec(
+        new AssertingCodec() {
+          @Override
+          public PostingsFormat getPostingsFormatForField(String field) {
+            return pf;
+          }
+
+          @Override
+          public DocValuesFormat getDocValuesFormatForField(String field) {
+            return dv;
+          }
+        });
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+
+    // index some docs
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
+      doc.add(idField);
+      int numValues = random().nextInt(17);
+      // create a random list of strings
+      List<String> values = new ArrayList<>();
+      for (int v = 0; v < numValues; v++) {
+        values.add(valuesProducer.get());
+      }
+
+      // add in any order to the indexed field
+      ArrayList<String> unordered = new ArrayList<>(values);
+      Collections.shuffle(unordered, random());
+      for (String v : values) {
+        doc.add(newStringField("indexed", v, Field.Store.NO));
+      }
+
+      // add in any order to the dv field
+      ArrayList<String> unordered2 = new ArrayList<>(values);
+      Collections.shuffle(unordered2, random());
+      for (String v : unordered2) {
+        doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
+      }
+
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+
+    // delete some docs
+    int numDeletions = random().nextInt(numDocs / 10);
+    for (int i = 0; i < numDeletions; i++) {
+      int id = random().nextInt(numDocs);
+      writer.deleteDocuments(new Term("id", Integer.toString(id)));
+    }
+
+    // compare per-segment
+    DirectoryReader ir = writer.getReader();
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      Terms terms = r.terms("indexed");
+      if (terms != null) {
+        SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
+        assertEquals(terms.size(), ssdv.getValueCount());
+        TermsEnum expected = terms.iterator();
+        TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
+        assertEquals(terms.size(), expected, actual);
+
+        doTestSortedSetEnumAdvanceIndependently(ssdv);
+      }
+    }
+    ir.close();
+
+    writer.forceMerge(1);
+
+    // now compare again after the merge
+    ir = writer.getReader();
+    LeafReader ar = getOnlyLeafReader(ir);
+    Terms terms = ar.terms("indexed");
+    if (terms != null) {
+      assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
+      TermsEnum expected = terms.iterator();
+      TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
+      assertEquals(terms.size(), expected, actual);
+    }
+    ir.close();
+
+    writer.close();
+    dir.close();
+  }
+
+  private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
+    BytesRef ref;
+
+    // sequential next() through all terms
+    while ((ref = expected.next()) != null) {
+      assertEquals(ref, actual.next());
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+    assertNull(actual.next());
+
+    // sequential seekExact(ord) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      actual.seekExact(i);
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+
+    // sequential seekExact(BytesRef) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      assertTrue(actual.seekExact(expected.term()));
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+
+    // sequential seekCeil(BytesRef) through all terms
+    for (long i = 0; i < numOrds; i++) {
+      expected.seekExact(i);
+      assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+
+    // random seekExact(ord)
+    for (long i = 0; i < numOrds; i++) {
+      long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+      expected.seekExact(randomOrd);
+      actual.seekExact(randomOrd);
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+
+    // random seekExact(BytesRef)
+    for (long i = 0; i < numOrds; i++) {
+      long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
+      expected.seekExact(randomOrd);
+      actual.seekExact(expected.term());
+      assertEquals(expected.ord(), actual.ord());
+      assertEquals(expected.term(), actual.term());
+    }
+
+    // random seekCeil(BytesRef)
+    for (long i = 0; i < numOrds; i++) {
+      BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
+      SeekStatus expectedStatus = expected.seekCeil(target);
+      assertEquals(expectedStatus, actual.seekCeil(target));
+      if (expectedStatus != SeekStatus.END) {
+        assertEquals(expected.ord(), actual.ord());
+        assertEquals(expected.term(), actual.term());
+      }
+    }
+  }
+
+  @Slow
+  public void testSortedSetAroundBlockSize() throws IOException {
+    final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+    for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
+      final Directory dir = newDirectory();
+      IndexWriter w =
+          new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+      ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+      Document doc = new Document();
+      SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
+      doc.add(field1);
+      SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef());
+      doc.add(field2);
+      for (int i = 0; i < maxDoc; ++i) {
+        BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
+        BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
+        field1.setBytesValue(s1);
+        field2.setBytesValue(s2);
+        w.addDocument(doc);
+        Set<BytesRef> set = new TreeSet<>(Arrays.asList(s1, s2));
+        out.writeVInt(set.size());
+        for (BytesRef ref : set) {
+          out.writeVInt(ref.length);
+          out.writeBytes(ref.bytes, ref.offset, ref.length);
+        }
+      }
+      w.forceMerge(1);
+      DirectoryReader r = DirectoryReader.open(w);
+      w.close();
+      LeafReader sr = getOnlyLeafReader(r);
+      assertEquals(maxDoc, sr.maxDoc());
+      SortedSetDocValues values = sr.getSortedSetDocValues("sset");
+      assertNotNull(values);
+      ByteBuffersDataInput in = new ByteBuffersDataInput(out.toBufferList());
+      BytesRefBuilder b = new BytesRefBuilder();
+      for (int i = 0; i < maxDoc; ++i) {
+        assertEquals(i, values.nextDoc());
+        final int numValues = in.readVInt();
+
+        for (int j = 0; j < numValues; ++j) {
+          b.setLength(in.readVInt());
+          b.grow(b.length());
+          in.readBytes(b.bytes(), 0, b.length());
+          assertEquals(b.get(), values.lookupOrd(values.nextOrd()));
+        }
+
+        assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd());
+      }
+      r.close();
+      dir.close();
+    }
+  }
+
+  @Slow
+  public void testSortedNumericAroundBlockSize() throws IOException {
+    final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+    for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
+      final Directory dir = newDirectory();
+      IndexWriter w =
+          new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+      ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+      Document doc = new Document();
+      SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
+      doc.add(field1);
+      SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L);
+      doc.add(field2);
+      for (int i = 0; i < maxDoc; ++i) {
+        long s1 = random().nextInt(100);
+        long s2 = random().nextInt(100);
+        field1.setLongValue(s1);
+        field2.setLongValue(s2);
+        w.addDocument(doc);
+        out.writeVLong(Math.min(s1, s2));
+        out.writeVLong(Math.max(s1, s2));
+      }
+      w.forceMerge(1);
+      DirectoryReader r = DirectoryReader.open(w);
+      w.close();
+      LeafReader sr = getOnlyLeafReader(r);
+      assertEquals(maxDoc, sr.maxDoc());
+      SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
+      assertNotNull(values);
+      ByteBuffersDataInput in = new ByteBuffersDataInput(out.toBufferList());
+      for (int i = 0; i < maxDoc; ++i) {
+        assertEquals(i, values.nextDoc());
+        assertEquals(2, values.docValueCount());
+        assertEquals(in.readVLong(), values.nextValue());
+        assertEquals(in.readVLong(), values.nextValue());
+      }
+      r.close();
+      dir.close();
+    }
+  }
+
+  @Slow
+  public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3));
+  }
+
+  @Slow
+  public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2));
+  }
+
+  @Slow
+  public void testNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSparseNumericBlocksOfVariousBitsPerValue(1);
+  }
+
+  @Slow
+  public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble());
+  }
+
+  private static LongSupplier blocksOfVariousBPV() {
+    final long mul = TestUtil.nextInt(random(), 1, 100);
+    final long min = random().nextInt();
+    return new LongSupplier() {
+      int i = Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
+      int maxDelta;
+
+      @Override
+      public long getAsLong() {
+        if (i == Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE) {
+          maxDelta = 1 << random().nextInt(5);
+          i = 0;
+        }
+        i++;
+        return min + mul * random().nextInt(maxDelta);
+      }
+    };
+  }
+
+  private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts)
+      throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
+    conf.setRAMBufferSizeMB(-1);
+    conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+
+    final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE * 3);
+    final LongSupplier values = blocksOfVariousBPV();
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+
+      int valueCount = (int) counts.getAsLong();
+      long valueArray[] = new long[valueCount];
+      for (int j = 0; j < valueCount; j++) {
+        long value = values.getAsLong();
+        valueArray[j] = value;
+        doc.add(new SortedNumericDocValuesField("dv", value));
+      }
+      Arrays.sort(valueArray);
+      for (int j = 0; j < valueCount; j++) {
+        doc.add(new StoredField("stored", Long.toString(valueArray[j])));
+      }
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    writer.forceMerge(1);
+
+    writer.close();
+
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    TestUtil.checkReader(ir);
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv");
+      for (int i = 0; i < r.maxDoc(); i++) {
+        if (i > docValues.docID()) {
+          docValues.nextDoc();
+        }
+        String expected[] = r.document(i).getValues("stored");
+        if (i < docValues.docID()) {
+          assertEquals(0, expected.length);
+        } else {
+          String actual[] = new String[docValues.docValueCount()];
+          for (int j = 0; j < actual.length; j++) {
+            actual[j] = Long.toString(docValues.nextValue());
+          }
+          assertArrayEquals(expected, actual);
+        }
+      }
+    }
+    ir.close();
+    dir.close();
+  }
+
+  private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
+    conf.setRAMBufferSizeMB(-1);
+    conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    Document doc = new Document();
+    Field storedField = newStringField("stored", "", Field.Store.YES);
+    Field dvField = new NumericDocValuesField("dv", 0);
+    doc.add(storedField);
+    doc.add(dvField);
+
+    final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE * 3);
+    final LongSupplier longs = blocksOfVariousBPV();
+    for (int i = 0; i < numDocs; i++) {
+      if (random().nextDouble() > density) {
+        writer.addDocument(new Document());
+        continue;
+      }
+      long value = longs.getAsLong();
+      storedField.setStringValue(Long.toString(value));
+      dvField.setLongValue(value);
+      writer.addDocument(doc);
+    }
+
+    writer.forceMerge(1);
+
+    writer.close();
+
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    TestUtil.checkReader(ir);
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      NumericDocValues docValues = DocValues.getNumeric(r, "dv");
+      docValues.nextDoc();
+      for (int i = 0; i < r.maxDoc(); i++) {
+        String storedValue = r.document(i).get("stored");
+        if (storedValue == null) {
+          assertTrue(docValues.docID() > i);
+        } else {
+          assertEquals(i, docValues.docID());
+          assertEquals(Long.parseLong(storedValue), docValues.longValue());
+          docValues.nextDoc();
+        }
+      }
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, docValues.docID());
+    }
+    ir.close();
+    dir.close();
+  }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70NormsFormat.java
new file mode 100644
index 0000000..5530235
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene70/TestLucene70NormsFormat.java
@@ -0,0 +1,28 @@
+/*
+ * 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.backward_codecs.lucene70;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseNormsFormatTestCase;
+
+/** Tests Lucene70NormsFormat */
+public class TestLucene70NormsFormat extends BaseNormsFormatTestCase {
+  @Override
+  protected Codec getCodec() {
+    return new Lucene70RWCodec();
+  }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java
index ea15562..9a3fb74 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestBackwardsCompatibility.java
@@ -386,7 +386,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     return oldSortedNames;
   }
 
-  final String[] unsupportedNames = {
+  static final String[] unsupportedNames = {
     "1.9.0-cfs",
     "1.9.0-nocfs",
     "2.0.0-cfs",
@@ -586,6 +586,20 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     "7.7.3-nocfs"
   };
 
+  static final int MIN_BINARY_SUPPORTED_MAJOR = Version.MIN_SUPPORTED_MAJOR - 1;
+
+  static final String[] binarySupportedNames;
+
+  static {
+    ArrayList<String> list = new ArrayList<>();
+    for (String name : unsupportedNames) {
+      if (name.startsWith(MIN_BINARY_SUPPORTED_MAJOR + ".")) {
+        list.add(name);
+      }
+    }
+    binarySupportedNames = list.toArray(new String[0]);
+  }
+
   // TODO: on 6.0.0 release, gen the single segment indices and add here:
   static final String[] oldSingleSegmentNames = {};
 
@@ -864,10 +878,10 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
       CheckIndex checker = new CheckIndex(dir);
       checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
       CheckIndex.Status indexStatus = checker.checkIndex();
-      assertFalse(indexStatus.clean);
       if (unsupportedNames[i].startsWith("7.")) {
-        assertTrue(bos.toString(IOUtils.UTF_8).contains("Could not load codec 'Lucene70'"));
+        assertTrue(indexStatus.clean);
       } else {
+        assertFalse(indexStatus.clean);
         assertTrue(
             bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName()));
       }
@@ -991,7 +1005,15 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
 
   public void testSearchOldIndex() throws IOException {
     for (String name : oldNames) {
-      searchIndex(oldIndexDirs.get(name), name);
+      searchIndex(oldIndexDirs.get(name), name, Version.MIN_SUPPORTED_MAJOR);
+    }
+
+    for (String name : binarySupportedNames) {
+      Path oldIndexDir = createTempDir(name);
+      TestUtil.unzip(getDataInputStream("unsupported." + name + ".zip"), oldIndexDir);
+      try (BaseDirectoryWrapper dir = newFSDirectory(oldIndexDir)) {
+        searchIndex(dir, name, MIN_BINARY_SUPPORTED_MAJOR);
+      }
     }
   }
 
@@ -1025,11 +1047,12 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     }
   }
 
-  public void searchIndex(Directory dir, String oldName) throws IOException {
+  public void searchIndex(Directory dir, String oldName, int minIndexMajorVersion)
+      throws IOException {
     // QueryParser parser = new QueryParser("contents", new MockAnalyzer(random));
     // Query query = parser.parse("handle:1");
-
-    IndexReader reader = DirectoryReader.open(dir);
+    IndexCommit indexCommit = DirectoryReader.listCommits(dir).get(0);
+    IndexReader reader = DirectoryReader.open(indexCommit, minIndexMajorVersion);
     IndexSearcher searcher = newSearcher(reader);
 
     TestUtil.checkIndex(dir);
@@ -2007,30 +2030,17 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
               .contains(
                   "only supports reading from version " + Version.LATEST.major + " upwards."));
       // now open with allowed min version
-      StandardDirectoryReader.open(commit, Version.LATEST.major - 1).close();
+      StandardDirectoryReader.open(commit, Version.MIN_SUPPORTED_MAJOR).close();
     }
   }
 
   public void testReadNMinusTwoCommit() throws IOException {
-    for (String name : this.unsupportedNames) {
-      if (name.startsWith(Version.MIN_SUPPORTED_MAJOR - 1 + ".")) {
-        Path oldIndexDir = createTempDir(name);
-        TestUtil.unzip(getDataInputStream("unsupported." + name + ".zip"), oldIndexDir);
-        try (BaseDirectoryWrapper dir = newFSDirectory(oldIndexDir)) {
-          // don't checkindex, we don't have the codecs yet
-          dir.setCheckIndexOnClose(false);
-          IllegalArgumentException iae =
-              expectThrows(IllegalArgumentException.class, () -> DirectoryReader.listCommits(dir));
-          // TODO fix this once we have the codec for 7.0 recreated
-          assertEquals(
-              "Could not load codec 'Lucene70'. Did you forget to add lucene-backward-codecs.jar?",
-              iae.getMessage());
-          IllegalArgumentException ex =
-              expectThrows(IllegalArgumentException.class, () -> DirectoryReader.listCommits(dir));
-          assertEquals(
-              "Could not load codec 'Lucene70'. Did you forget to add lucene-backward-codecs.jar?",
-              ex.getMessage());
-        }
+    for (String name : binarySupportedNames) {
+      Path oldIndexDir = createTempDir(name);
+      TestUtil.unzip(getDataInputStream("unsupported." + name + ".zip"), oldIndexDir);
+      try (BaseDirectoryWrapper dir = newFSDirectory(oldIndexDir)) {
+        IndexCommit commit = DirectoryReader.listCommits(dir).get(0);
+        StandardDirectoryReader.open(commit, MIN_BINARY_SUPPORTED_MAJOR).close();
       }
     }
   }