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

[1/6] lucene-solr:master: LUCENE-7475: Make norms sparse.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 28d187acd -> 9128bdbaf


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
new file mode 100644
index 0000000..00cd5ec
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+import static org.apache.lucene.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);
+    } else if (numDocsWithValue == maxDoc) {
+      meta.writeLong(-1);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      values = normsProducer.getNorms(field);
+      SparseDISI.writeBitSet(values, maxDoc, data);
+    }
+
+    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();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java
new file mode 100644
index 0000000..7e70b24
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+import java.io.IOException;
+
+import 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><tt>.nvd</tt>: Norms data</li>
+ *   <li><tt>.nvm</tt>: Norms metadata</li>
+ * </ol>
+ * <ol>
+ *   <li><a name="nvm"></a>
+ *   <p>The Norms metadata or .nvm file.</p>
+ *   <p>For each norms field, this stores metadata, such as the offset into the 
+ *      Norms data (.nvd)</p>
+ *   <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
+ *     <li>Entry --&gt; FieldNumber, DocsWithFieldAddress, NumDocsWithField, BytesPerNorm, NormsAddress</li>
+ *     <li>FieldNumber --&gt; {@link DataOutput#writeInt Int32}</li>
+ *     <li>DocsWithFieldAddress --&gt; {@link DataOutput#writeLong Int64}</li>
+ *     <li>NumDocsWithField --&gt; {@link DataOutput#writeInt Int32}</li>
+ *     <li>BytesPerNorm --&gt; {@link DataOutput#writeByte byte}</li>
+ *     <li>NormsAddress --&gt; {@link DataOutput#writeLong Int64}</li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </ul>
+ *   <p>FieldNumber of -1 indicates the end of metadata.</p>
+ *   <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>
+ *   <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>
+ *   <li><a name="nvd"></a>
+ *   <p>The Norms data or .nvd file.</p>
+ *   <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
+ *   <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
+ *     <li>DocsWithFieldData --&gt; Bit set of MaxDoc bits</li>
+ *     <li>NormsData --&gt; {@link DataOutput#writeByte(byte) byte}<sup>NumDocsWithField * BytesPerValue</sup></li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </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;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java
new file mode 100644
index 0000000..ee96c15
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsProducer.java
@@ -0,0 +1,271 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+import static org.apache.lucene.codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
+import static org.apache.lucene.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 {
+  // metadata maps (just file pointers and minimal stuff)
+  private final Map<Integer,NormsEntry> norms = new HashMap<>();
+  private final IndexInput data;
+  private final int maxDoc;
+
+  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);
+      }
+    }
+  }
+
+  static class NormsEntry {
+    byte bytesPerNorm;
+    long docsWithFieldOffset;
+    int numDocsWithField;
+    long normsOffset;
+  }
+
+  static abstract class LongValues {
+    abstract long get(int index) throws IOException;
+  }
+
+  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.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);
+    }
+  }
+
+  @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
+      final LongValues normValues = getNormValues(entry);
+      return new NumericDocValues() {
+
+        int doc = -1;
+
+        @Override
+        public long longValue() throws IOException {
+          return normValues.get(doc);
+        }
+
+        @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 long cost() {
+          return maxDoc;
+        }
+
+      };
+    } else {
+      // sparse
+      final LongValues normValues = getNormValues(entry);
+      final SparseDISI disi;
+      synchronized (data) {
+        disi = new SparseDISI(maxDoc, data, entry.docsWithFieldOffset, entry.numDocsWithField);
+      }
+      return new NumericDocValues() {
+
+        @Override
+        public int advance(int target) throws IOException {
+          return disi.advance(target);
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          return disi.nextDoc();
+        }
+
+        @Override
+        public int docID() {
+          return disi.docID();
+        }
+
+        @Override
+        public long cost() {
+          return entry.numDocsWithField;
+        }
+
+        @Override
+        public long longValue() throws IOException {
+          return normValues.get(disi.index());
+        }
+      };
+    }
+  }
+
+  private LongValues getNormValues(NormsEntry entry) throws IOException {
+    if (entry.bytesPerNorm == 0) {
+      return new LongValues() {
+        @Override
+        long get(int index) {
+          return entry.normsOffset;
+        }
+      };
+    } else {
+      RandomAccessInput slice;
+      synchronized (data) {
+        slice = data.randomAccessSlice(entry.normsOffset, entry.numDocsWithField * (long) entry.bytesPerNorm);
+      }
+      switch (entry.bytesPerNorm) {
+        case 1:
+          return new LongValues() {
+            @Override
+            long get(int index) throws IOException {
+              return slice.readByte(index);
+            }
+          };
+        case 2:
+          return new LongValues() {
+            @Override
+            long get(int index) throws IOException {
+              return slice.readShort(((long) index) << 1);
+            }
+          };
+        case 4:
+          return new LongValues() {
+            @Override
+            long get(int index) throws IOException {
+              return slice.readInt(((long) index) << 2);
+            }
+          };
+        case 8:
+          return new LongValues() {
+            @Override
+            long get(int index) throws IOException {
+              return slice.readLong(((long) 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() + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene70/SparseDISI.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/SparseDISI.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/SparseDISI.java
new file mode 100644
index 0000000..af71b9e
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/SparseDISI.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RandomAccessInput;
+
+final class SparseDISI extends DocIdSetIterator {
+
+  static void writeBitSet(DocIdSetIterator it, int maxDoc, IndexOutput out) throws IOException {
+    int currentIndex = 0;
+    long currentBits = 0;
+    for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+      final int index = doc >>> 6;
+      if (index > currentIndex) {
+        out.writeLong(currentBits);
+        for (int i = currentIndex + 1; i < index; ++i) {
+          out.writeLong(0L);
+        }
+        currentIndex = index;
+        currentBits = 0L;
+      }
+      currentBits |= 1L << doc;
+    }
+
+    out.writeLong(currentBits);
+    final int maxIndex = (maxDoc - 1) >>> 6;
+    for (int i = currentIndex + 1; i <= maxIndex; ++i) {
+      out.writeLong(0L);
+    }
+  }
+
+  final int maxDoc;
+  final int numWords;
+  final long cost;
+  final RandomAccessInput slice;
+  int doc = -1;
+  int wordIndex = -1;
+  long word;
+  int index = -1;
+
+  SparseDISI(int maxDoc, IndexInput in, long offset, long cost) throws IOException {
+    this.maxDoc = maxDoc;
+    this.numWords = (int) ((maxDoc + 63L) >>> 6);
+    this.slice = in.randomAccessSlice(offset, numWords * 8L);
+    this.cost = cost;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    if (target >= maxDoc) {
+      return doc = NO_MORE_DOCS;
+    }
+
+    final int targetWordIndex = target >>> 6;
+    for (int i = wordIndex + 1; i <= targetWordIndex; ++i) {
+      word = slice.readLong(i << 3);
+      index += Long.bitCount(word);
+    }
+    wordIndex = targetWordIndex;
+
+    long leftBits = word >>> target;
+    if (leftBits != 0L) {
+      return doc = target + Long.numberOfTrailingZeros(leftBits);
+    }
+
+    while (++wordIndex < numWords) {
+      word = slice.readLong(wordIndex << 3);
+      if (word != 0) {
+        index += Long.bitCount(word);
+        return doc = (wordIndex << 6) + Long.numberOfTrailingZeros(word);
+      }
+    }
+
+    return doc = NO_MORE_DOCS;
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return advance(doc + 1);
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public long cost() {
+    return cost;
+  }
+
+  public int index() {
+    return index - Long.bitCount(word >>> doc) + 1;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
index 77492ad..9b432f7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
@@ -163,7 +163,7 @@
  * all documents omit position data.
  * </li>
  * <li>
- * {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}. 
+ * {@link org.apache.lucene.codecs.lucene70.Lucene70NormsFormat Normalization factors}. 
  * For each field in each document, a value is stored
  * that is multiplied into the score for hits on that field.
  * </li>
@@ -278,12 +278,12 @@
  * <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
  * </tr>
  * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
+ * <td>{@link org.apache.lucene.codecs.lucene70.Lucene70NormsFormat Norms}</td>
  * <td>.nvd, .nvm</td>
  * <td>Encodes length and boost factors for docs and fields</td>
  * </tr>
  * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
+ * <td>{@link org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat Per-Document Values}</td>
  * <td>.dvd, .dvm</td>
  * <td>Encodes additional scoring factors or other per-document information.</td>
  * </tr>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
index e941911..e2ece54 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -665,8 +665,17 @@ final class DefaultIndexingChain extends DocConsumer {
     }
 
     public void finish() throws IOException {
-      if (fieldInfo.omitsNorms() == false && invertState.length != 0) {
-        norms.addValue(docState.docID, similarity.computeNorm(invertState));
+      if (fieldInfo.omitsNorms() == false) {
+        long normValue;
+        if (invertState.length == 0) {
+          // the field exists in this document, but it did not have
+          // any indexed tokens, so we assign a default value of zero
+          // to the norm
+          normValue = 0;
+        } else {
+          normValue = similarity.computeNorm(invertState);
+        }
+        norms.addValue(docState.docID, normValue);
       }
 
       termsHashPerField.finish();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
index c444661..46b8c1c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
@@ -21,7 +21,10 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
@@ -29,29 +32,34 @@ import org.apache.lucene.util.packed.PackedLongValues;
  *  segment flushes. */
 class NormValuesWriter {
 
-  private final static long MISSING = 0L;
-
+  private FixedBitSet docsWithField;
   private PackedLongValues.Builder pending;
   private final Counter iwBytesUsed;
   private long bytesUsed;
   private final FieldInfo fieldInfo;
+  private int lastDocID = -1;
 
   public NormValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+    docsWithField = new FixedBitSet(64);
     pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
-    bytesUsed = pending.ramBytesUsed();
+    bytesUsed = pending.ramBytesUsed() + docsWithField.ramBytesUsed();
     this.fieldInfo = fieldInfo;
     this.iwBytesUsed = iwBytesUsed;
     iwBytesUsed.addAndGet(bytesUsed);
   }
 
   public void addValue(int docID, long value) {
-    // Fill in any holes:
-    for (int i = (int)pending.size(); i < docID; ++i) {
-      pending.add(MISSING);
+    if (docID <= lastDocID) {
+      throw new IllegalArgumentException("Norm for \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
     }
 
     pending.add(value);
+    docsWithField = FixedBitSet.ensureCapacity(docsWithField, docID);
+    docsWithField.set(docID);
+
     updateBytesUsed();
+
+    lastDocID = docID;
   }
 
   private void updateBytesUsed() {
@@ -65,7 +73,6 @@ class NormValuesWriter {
 
   public void flush(SegmentWriteState state, NormsConsumer normsConsumer) throws IOException {
 
-    final int maxDoc = state.segmentInfo.maxDoc();
     final PackedLongValues values = pending.build();
 
     normsConsumer.addNormsField(fieldInfo,
@@ -75,7 +82,7 @@ class NormValuesWriter {
                                    if (fieldInfo != NormValuesWriter.this.fieldInfo) {
                                      throw new IllegalArgumentException("wrong fieldInfo");
                                    }
-                                   return new BufferedNorms(maxDoc, values);
+                                   return new BufferedNorms(values, docsWithField);
                                   }
 
                                   @Override
@@ -98,36 +105,28 @@ class NormValuesWriter {
   // iterates over the values we have in ram
   private static class BufferedNorms extends NumericDocValues {
     final PackedLongValues.Iterator iter;
-    final int size;
-    final int maxDoc;
-    private int docID = -1;
+    final DocIdSetIterator docsWithField;
     private long value;
-    
-    BufferedNorms(int maxDoc, PackedLongValues values) {
-      this.maxDoc = maxDoc;
+
+    BufferedNorms(PackedLongValues values, FixedBitSet docsWithFields) {
       this.iter = values.iterator();
-      this.size = (int) values.size();
+      this.docsWithField = new BitSetIterator(docsWithFields, values.size());
     }
-    
+
     @Override
     public int docID() {
-      return docID;
+      return docsWithField.docID();
     }
 
     @Override
-    public int nextDoc() {
-      docID++;
-      if (docID == maxDoc) {
-        docID = NO_MORE_DOCS;
-      }
-      if (docID < size) {
+    public int nextDoc() throws IOException {
+      int docID = docsWithField.nextDoc();
+      if (docID != NO_MORE_DOCS) {
         value = iter.next();
-      } else {
-        value = MISSING;
       }
       return docID;
     }
-    
+
     @Override
     public int advance(int target) {
       throw new UnsupportedOperationException();
@@ -135,7 +134,7 @@ class NormValuesWriter {
 
     @Override
     public long cost() {
-      return maxDoc;
+      return docsWithField.cost();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
----------------------------------------------------------------------
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 85aa3a7..773c168 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -13,5 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene62.Lucene62Codec
 org.apache.lucene.codecs.lucene70.Lucene70Codec

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
----------------------------------------------------------------------
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
index 1161f03..20463c5 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
@@ -13,5 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat
 org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
deleted file mode 100644
index c87c51f..0000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene53;
-
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene70.Lucene70Codec;
-import org.apache.lucene.index.BaseNormsFormatTestCase;
-
-/**
- * Tests Lucene53NormsFormat
- */
-public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
-  private final Codec codec = new Lucene70Codec();
-  
-  @Override
-  protected Codec getCodec() {
-    return codec;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
deleted file mode 100644
index c6ca201..0000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
+++ /dev/null
@@ -1,612 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene54;
-
-
-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 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.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValues;
-import org.apache.lucene.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValuesRandomAccessWrapper;
-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.SeekStatus;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.LongValues;
-import org.apache.lucene.util.TestUtil;
-
-/**
- * Tests Lucene54DocValuesFormat
- */
-public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
-  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene54DocValuesFormat());
-
-  @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), 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), 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), 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), 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), 1, 500);
-    }
-  }
-
-  @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, int minLength, int maxLength) 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 Lucene54DocValuesFormat();
-    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);
-      final int length = TestUtil.nextInt(random(), minLength, maxLength);
-      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(TestUtil.randomSimpleString(random(), minLength, length));
-      }
-      
-      // 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());
-      }
-    }
-  }
-
-  public void testSparseLongValues() throws IOException {
-    final int iters = atLeast(5);
-    for (int iter = 0; iter < iters; ++iter) {
-      final int numDocs = TestUtil.nextInt(random(), 0, 100);
-      final int[] docIds = new int[numDocs];
-      final long[] values = new long[numDocs];
-      final int maxDoc;
-      if (numDocs == 0) {
-        maxDoc = 1 + random().nextInt(10);
-      } else {
-        docIds[0] = random().nextInt(10);
-        for (int i = 1; i < docIds.length; ++i) {
-          docIds[i] = docIds[i - 1] + 1 + random().nextInt(100);
-        }
-        maxDoc = docIds[numDocs - 1] + 1 + random().nextInt(10);
-      }
-      for (int i = 0; i < values.length; ++i) {
-        values[i] = random().nextLong();
-      }
-      final long missingValue = random().nextLong();
-      final LongValues docIdsValues = new LongValues() {
-        @Override
-        public long get(long index) {
-          return docIds[Math.toIntExact(index)];
-        }
-      };
-      final LongValues valuesValues = new LongValues() {
-        @Override
-        public long get(long index) {
-          return values[Math.toIntExact(index)];
-        }
-      };
-      final SparseNumericDocValues sparseValues = new SparseNumericDocValues(numDocs, docIdsValues, valuesValues);
-
-      // sequential access
-      assertEquals(-1, sparseValues.docID());
-      for (int i = 0; i < docIds.length; ++i) {
-        assertEquals(docIds[i], sparseValues.nextDoc());
-      }
-      assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc());
-
-      // advance
-      for (int i = 0; i < 2000; ++i) {
-        final int target = TestUtil.nextInt(random(), 0, maxDoc);
-        int index = Arrays.binarySearch(docIds, target);
-        if (index < 0) {
-          index = -1 - index;
-        }
-        sparseValues.reset();
-        if (index > 0) {
-          assertEquals(docIds[index - 1], sparseValues.advance(Math.toIntExact(docIds[index - 1])));
-        }
-        if (index == docIds.length) {
-          assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.advance(target));
-        } else {
-          assertEquals(docIds[index], sparseValues.advance(target));
-        }
-      }
-
-      final SparseNumericDocValuesRandomAccessWrapper raWrapper = new SparseNumericDocValuesRandomAccessWrapper(sparseValues, missingValue);
-
-      // random-access
-      for (int i = 0; i < 2000; ++i) {
-        final int docId = TestUtil.nextInt(random(), 0, maxDoc - 1);
-        final int idx = Arrays.binarySearch(docIds, docId);
-        final long value = raWrapper.get(docId);
-        if (idx >= 0) {
-          assertEquals(values[idx], value);
-        } else {
-          assertEquals(missingValue, value);
-        }
-      }
-
-      // sequential access
-      for (int docId = 0; docId < maxDoc; docId += random().nextInt(3)) {
-        final int idx = Arrays.binarySearch(docIds, docId);
-        final long value = raWrapper.get(docId);
-        if (idx >= 0) {
-          assertEquals(values[idx], value);
-        } else {
-          assertEquals(missingValue, value);
-        }
-      }
-    }
-  }
-
-  @Slow
-  public void testSortedSetAroundBlockSize() throws IOException {
-    final int frontier = 1 << Lucene54DocValuesFormat.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()));
-      RAMFile buffer = new RAMFile();
-      RAMOutputStream out = new RAMOutputStream(buffer, false);
-      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);
-        }
-      }
-      out.close();
-      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);
-      RAMInputStream in = new RAMInputStream("", buffer);
-      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 << Lucene54DocValuesFormat.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()));
-      RAMFile buffer = new RAMFile();
-      RAMOutputStream out = new RAMOutputStream(buffer, false);
-      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));
-      }
-      out.close();
-      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);
-      RAMInputStream in = new RAMInputStream("", buffer);
-      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();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java
new file mode 100644
index 0000000..cc07cee
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70NormsFormat.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
+import org.apache.lucene.index.BaseNormsFormatTestCase;
+
+/**
+ * Tests Lucene70NormsFormat
+ */
+public class TestLucene70NormsFormat extends BaseNormsFormatTestCase {
+  private final Codec codec = new Lucene70Codec();
+  
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestSparseDISI.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestSparseDISI.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestSparseDISI.java
new file mode 100644
index 0000000..1911bd0
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestSparseDISI.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+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 TestSparseDISI extends LuceneTestCase {
+
+  public void testRandom() throws IOException {
+    try (Directory dir = newDirectory()) {
+      for (int i = 0; i < 1000; ++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, 10));
+    final int numDocs = TestUtil.nextInt(random(), 1, 1000);
+    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);
+    }
+
+    try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) {
+      SparseDISI.writeBitSet(new BitSetIterator(set, docs.size()), maxDoc, out);
+    }
+
+    try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
+      SparseDISI disi = new SparseDISI(maxDoc, in, 0L, docs.size());
+      BitSetIterator disi2 = new BitSetIterator(set, docs.size());
+      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, 20, maxStep, maxStep * 10}) {
+      try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) {
+        SparseDISI disi = new SparseDISI(maxDoc, in, 0L, docs.size());
+        BitSetIterator disi2 = new BitSetIterator(set, docs.size());
+        while (true) {
+          int target = disi2.docID() + step;
+          int doc = disi2.advance(target);
+          assertEquals(doc, disi.advance(target));
+          if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+            break;
+          }
+          int index = Collections.binarySearch(docs, doc);
+          assertEquals(index, disi.index());
+        }
+      }
+    }
+
+    dir.deleteFile("foo");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
index a51f1d9..52038bc 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java
@@ -23,6 +23,7 @@ import java.util.Random;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.TermStatistics;
@@ -186,4 +187,24 @@ public class TestNorms extends LuceneTestCase {
       throw new UnsupportedOperationException();
     }
   } 
+
+  public void testEmptyValueVsNoValue() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig cfg = newIndexWriterConfig().setMergePolicy(newLogMergePolicy());
+    IndexWriter w = new IndexWriter(dir, cfg);
+    Document doc = new Document();
+    w.addDocument(doc);
+    doc.add(newTextField("foo", "", Store.NO));
+    w.addDocument(doc);
+    w.forceMerge(1);
+    IndexReader reader = DirectoryReader.open(w);
+    w.close();
+    LeafReader leafReader = getOnlyLeafReader(reader);
+    NumericDocValues normValues = leafReader.getNormValues("foo");
+    assertNotNull(normValues);
+    assertEquals(1, normValues.nextDoc()); // doc 0 does not have norms
+    assertEquals(0, normValues.longValue());
+    reader.close();
+    dir.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
index dc0636d..9ca13dd 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
@@ -30,9 +30,11 @@ import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.TestUtil;
 
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
@@ -46,12 +48,17 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
  * if there is some bug in a given NormsFormat that this
  * test fails to catch then this test needs to be improved! */
 public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCase {
-  
+
+  /** Whether the codec supports sparse values. */
+  protected boolean codecSupportsSparsity() {
+    return true;
+  }
+
   public void testByteRange() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
@@ -59,12 +66,26 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testSparseByteRange() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+        }
+      });
+    }
+  }
+
   public void testShortRange() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return TestUtil.nextLong(r, Short.MIN_VALUE, Short.MAX_VALUE);
@@ -72,12 +93,26 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testSparseShortRange() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return TestUtil.nextLong(r, Short.MIN_VALUE, Short.MAX_VALUE);
+        }
+      });
+    }
+  }
+
   public void testLongRange() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return TestUtil.nextLong(r, Long.MIN_VALUE, Long.MAX_VALUE);
@@ -85,12 +120,26 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testSparseLongRange() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return TestUtil.nextLong(r, Long.MIN_VALUE, Long.MAX_VALUE);
+        }
+      });
+    }
+  }
+
   public void testFullLongRange() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           int thingToDo = r.nextInt(3);
@@ -103,12 +152,31 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testSparseFullLongRange() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          int thingToDo = r.nextInt(3);
+          switch (thingToDo) {
+            case 0: return Long.MIN_VALUE;
+            case 1: return Long.MAX_VALUE;
+            default:  return TestUtil.nextLong(r, Long.MIN_VALUE, Long.MAX_VALUE);
+          }
+        }
+      });
+    }
+  }
+
   public void testFewValues() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return r.nextBoolean() ? 20 : 3;
@@ -116,12 +184,26 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testFewSparseValues() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return r.nextBoolean() ? 20 : 3;
+        }
+      });
+    }
+  }
+
   public void testFewLargeValues() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return r.nextBoolean() ? 1000000L : -5000;
@@ -129,11 +211,25 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testFewSparseLargeValues() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return r.nextBoolean() ? 1000000L : -5000;
+        }
+      });
+    }
+  }
+
   public void testAllZeros() throws Exception {
     int iterations = atLeast(1);
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return 0;
@@ -141,12 +237,25 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
-  public void testSparse() throws Exception {
+
+  public void testSparseAllZeros() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    for (int i = 0; i < iterations; i++) {
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return 0;
+        }
+      });
+    }
+  }
+
+  public void testMostZeros() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : 0;
@@ -160,7 +269,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
       final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : commonValue;
@@ -168,14 +277,29 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testSparseOutliers() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : commonValue;
+        }
+      });
+    }
+  }
+
   public void testOutliers2() throws Exception {
     int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; i++) {
       final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
       final long uncommonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
-      doTestNormsVersusDocValues(new LongProducer() {
+      doTestNormsVersusDocValues(1, new LongProducer() {
         @Override
         long next() {
           return r.nextInt(100) == 0 ? uncommonValue : commonValue;
@@ -183,7 +307,23 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       });
     }
   }
-  
+
+  public void testSparseOutliers2() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; i++) {
+      final long commonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+      final long uncommonValue = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+      doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+        @Override
+        long next() {
+          return r.nextInt(100) == 0 ? uncommonValue : commonValue;
+        }
+      });
+    }
+  }
+
   public void testNCommon() throws Exception {
     final Random r = random();
     final int N = TestUtil.nextInt(r, 2, 15);
@@ -196,14 +336,35 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
     for (int j = 0; j < numOtherValues; ++j) {
       otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
     }
-    doTestNormsVersusDocValues(new LongProducer() {
+    doTestNormsVersusDocValues(1, new LongProducer() {
       @Override
       long next() {
         return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
       }
     });
   }
-  
+
+  public void testSparseNCommon() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    final Random r = random();
+    final int N = TestUtil.nextInt(r, 2, 15);
+    final long[] commonValues = new long[N];
+    for (int j = 0; j < N; ++j) {
+      commonValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+    }
+    final int numOtherValues = TestUtil.nextInt(r, 2, 256 - N);
+    final long[] otherValues = new long[numOtherValues];
+    for (int j = 0; j < numOtherValues; ++j) {
+      otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+    }
+    doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+      @Override
+      long next() {
+        return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
+      }
+    });
+  }
+
   /**
    * a more thorough n-common that tests all low bpv
    */
@@ -224,7 +385,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
         for (int j = 0; j < numOtherValues; ++j) {
           otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
         }
-        doTestNormsVersusDocValues(new LongProducer() {
+        doTestNormsVersusDocValues(1, new LongProducer() {
           @Override
           long next() {
             return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
@@ -233,17 +394,62 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       }
     }
   }
-  
-  private void doTestNormsVersusDocValues(LongProducer longs) throws Exception {
+
+  /**
+   * a more thorough n-common that tests all low bpv and sparse docs
+   */
+  @Nightly
+  public void testSparseNCommonBig() throws Exception {
+    assumeTrue("Requires sparse norms support", codecSupportsSparsity());
+    final int iterations = atLeast(1);
+    final Random r = random();
+    for (int i = 0; i < iterations; ++i) {
+      // 16 is 4 bpv, the max before we jump to 8bpv
+      for (int n = 2; n < 16; ++n) {
+        final int N = n;
+        final long[] commonValues = new long[N];
+        for (int j = 0; j < N; ++j) {
+          commonValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+        }
+        final int numOtherValues = TestUtil.nextInt(r, 2, 256 - N);
+        final long[] otherValues = new long[numOtherValues];
+        for (int j = 0; j < numOtherValues; ++j) {
+          otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+        }
+        doTestNormsVersusDocValues(random().nextDouble(), new LongProducer() {
+          @Override
+          long next() {
+            return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
+          }
+        });
+      }
+    }
+  }
+
+  private void doTestNormsVersusDocValues(double density, LongProducer longs) throws Exception {
     int numDocs = atLeast(500);
-    long norms[] = new long[numDocs];
-    for (int i = 0; i < numDocs; i++) {
+    final FixedBitSet docsWithField = new FixedBitSet(numDocs);
+    final int numDocsWithField = Math.max(1, (int) (density * numDocs));
+    if (numDocsWithField == numDocs) {
+      docsWithField.set(0, numDocs);
+    } else {
+      int i = 0;
+      while (i < numDocsWithField) {
+        int doc = random().nextInt(numDocs);
+        if (docsWithField.get(doc) == false) {
+          docsWithField.set(doc);
+          ++i;
+        }
+      }
+    }
+    long norms[] = new long[numDocsWithField];
+    for (int i = 0; i < numDocsWithField; i++) {
       norms[i] = longs.next();
     }
     
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
-    IndexWriterConfig conf = newIndexWriterConfig(analyzer);
+    IndexWriterConfig conf = newIndexWriterConfig(analyzer);conf.setMergePolicy(NoMergePolicy.INSTANCE);
     conf.setSimilarity(new CannedNormSimilarity(norms));
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
     Document doc = new Document();
@@ -254,12 +460,18 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
     doc.add(indexedField);
     doc.add(dvField);
     
-    for (int i = 0; i < numDocs; i++) {
+    for (int i = 0, j = 0; i < numDocs; i++) {
       idField.setStringValue(Integer.toString(i));
-      long value = norms[i];
-      dvField.setLongValue(value);
-      indexedField.setStringValue(Long.toString(value));
-      writer.addDocument(doc);
+      if (docsWithField.get(i) == false) {
+        Document doc2 = new Document();
+        doc2.add(idField);
+        writer.addDocument(doc2);
+      } else {
+        long value = norms[j++];
+        dvField.setLongValue(value);
+        indexedField.setStringValue(Long.toString(value));
+        writer.addDocument(doc);
+      }
       if (random().nextInt(31) == 0) {
         writer.commit();
       }
@@ -280,12 +492,14 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       LeafReader r = context.reader();
       NumericDocValues expected = r.getNumericDocValues("dv");
       NumericDocValues actual = r.getNormValues("indexed");
-      for (int i = 0; i < r.maxDoc(); i++) {
-        assertEquals(i, expected.nextDoc());
-        assertEquals(i, actual.nextDoc());
-        assertEquals("doc " + i, expected.longValue(), actual.longValue());
+      assertEquals(expected == null, actual == null);
+      if (expected != null) {
+        for (int d = expected.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = expected.nextDoc()) {
+          assertEquals(d, actual.nextDoc());
+          assertEquals("doc " + d, expected.longValue(), actual.longValue());
+        }
+        assertEquals(NO_MORE_DOCS, actual.nextDoc());
       }
-      assertEquals(NO_MORE_DOCS, expected.nextDoc());
     }
     ir.close();
     
@@ -297,10 +511,13 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
       LeafReader r = context.reader();
       NumericDocValues expected = r.getNumericDocValues("dv");
       NumericDocValues actual = r.getNormValues("indexed");
-      for (int i = 0; i < r.maxDoc(); i++) {
-        assertEquals(i, expected.nextDoc());
-        assertEquals(i, actual.nextDoc());
-        assertEquals("doc " + i, expected.longValue(), actual.longValue());
+      assertEquals(expected == null, actual == null);
+      if (expected != null) {
+        for (int d = expected.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = expected.nextDoc()) {
+          assertEquals(d, actual.nextDoc());
+          assertEquals("doc " + d, expected.longValue(), actual.longValue());
+        }
+        assertEquals(NO_MORE_DOCS, actual.nextDoc());
       }
     }
     
@@ -403,9 +620,13 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
     // be undead:
     NumericDocValues norms = MultiDocValues.getNormValues(r, "content");
     assertNotNull(norms);
-    for(int i=0;i<r.maxDoc();i++) {
-      assertEquals(i, norms.nextDoc());
-      assertEquals(0, norms.longValue());
+    if (codecSupportsSparsity()) {
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, norms.nextDoc());
+    } else {
+      for(int i=0;i<r.maxDoc();i++) {
+        assertEquals(i, norms.nextDoc());
+        assertEquals(0, norms.longValue());
+      }
     }
 
     r.close();


[5/6] lucene-solr:master: LUCENE-7475: Make norms sparse.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
new file mode 100644
index 0000000..9356aed
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
@@ -0,0 +1,1707 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene54;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesConsumer.NumberType;
+import org.apache.lucene.index.*;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
+
+import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
+
+/** reader for {@link Lucene54DocValuesFormat} */
+final class Lucene54DocValuesProducer extends DocValuesProducer implements Closeable {
+  private final Map<String,NumericEntry> numerics = new HashMap<>();
+  private final Map<String,BinaryEntry> binaries = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
+  private final Map<String,NumericEntry> ords = new HashMap<>();
+  private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
+  private final int numFields;
+  private final AtomicLong ramBytesUsed;
+  private final IndexInput data;
+  private final int maxDoc;
+
+  // memory-resident structures
+  private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
+  private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
+  private final Map<String,DirectMonotonicReader.Meta> directAddressesMeta = new HashMap<>();
+
+  private final boolean merging;
+
+  // clone for merge: when merging we don't do any instances.put()s
+  Lucene54DocValuesProducer(Lucene54DocValuesProducer original) throws IOException {
+    assert Thread.holdsLock(original);
+    numerics.putAll(original.numerics);
+    binaries.putAll(original.binaries);
+    sortedSets.putAll(original.sortedSets);
+    sortedNumerics.putAll(original.sortedNumerics);
+    ords.putAll(original.ords);
+    ordIndexes.putAll(original.ordIndexes);
+    numFields = original.numFields;
+    ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
+    data = original.data.clone();
+    maxDoc = original.maxDoc;
+
+    addressInstances.putAll(original.addressInstances);
+    reverseIndexInstances.putAll(original.reverseIndexInstances);
+    merging = true;
+  }
+
+  /** expert: instantiates a new reader */
+  Lucene54DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+    this.maxDoc = state.segmentInfo.maxDoc();
+    merging = false;
+    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+
+    int version = -1;
+    int numFields = -1;
+
+    // read in the entries from the metadata file.
+    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
+      Throwable priorE = null;
+      try {
+        version = CodecUtil.checkIndexHeader(in, metaCodec,
+                                        Lucene54DocValuesFormat.VERSION_START,
+                                        Lucene54DocValuesFormat.VERSION_CURRENT,
+                                        state.segmentInfo.getId(),
+                                        state.segmentSuffix);
+        numFields = readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
+      }
+    }
+
+    this.numFields = numFields;
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+    this.data = state.directory.openInput(dataName, state.context);
+    boolean success = false;
+    try {
+      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec,
+                                                 Lucene54DocValuesFormat.VERSION_START,
+                                                 Lucene54DocValuesFormat.VERSION_CURRENT,
+                                                 state.segmentInfo.getId(),
+                                                 state.segmentSuffix);
+      if (version != version2) {
+        throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data);
+      }
+
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(data);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.data);
+      }
+    }
+  }
+
+  private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
+    // sorted = binary + numeric
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    BinaryEntry b = readBinaryEntry(info, meta);
+    binaries.put(info.name, b);
+
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n = readNumericEntry(info, meta);
+    ords.put(info.name, n);
+  }
+
+  private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
+    // sortedset = binary + numeric (addresses) + ordIndex
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    BinaryEntry b = readBinaryEntry(info, meta);
+    binaries.put(info.name, b);
+
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n1 = readNumericEntry(info, meta);
+    ords.put(info.name, n1);
+
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n2 = readNumericEntry(info, meta);
+    ordIndexes.put(info.name, n2);
+  }
+
+  private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
+    // sortedset table = binary + ordset table + ordset index
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+
+    BinaryEntry b = readBinaryEntry(info, meta);
+    binaries.put(info.name, b);
+
+    if (meta.readVInt() != info.number) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+    }
+    NumericEntry n = readNumericEntry(info, meta);
+    ords.put(info.name, n);
+  }
+
+  private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int numFields = 0;
+    int fieldNumber = meta.readVInt();
+    while (fieldNumber != -1) {
+      numFields++;
+      FieldInfo info = infos.fieldInfo(fieldNumber);
+      if (info == null) {
+        // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
+      }
+      byte type = meta.readByte();
+      if (type == Lucene54DocValuesFormat.NUMERIC) {
+        numerics.put(info.name, readNumericEntry(info, meta));
+      } else if (type == Lucene54DocValuesFormat.BINARY) {
+        BinaryEntry b = readBinaryEntry(info, meta);
+        binaries.put(info.name, b);
+      } else if (type == Lucene54DocValuesFormat.SORTED) {
+        readSortedField(info, meta);
+      } else if (type == Lucene54DocValuesFormat.SORTED_SET) {
+        SortedSetEntry ss = readSortedSetEntry(meta);
+        sortedSets.put(info.name, ss);
+        if (ss.format == SORTED_WITH_ADDRESSES) {
+          readSortedSetFieldWithAddresses(info, meta);
+        } else if (ss.format == SORTED_SET_TABLE) {
+          readSortedSetFieldWithTable(info, meta);
+        } else if (ss.format == SORTED_SINGLE_VALUED) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene54DocValuesFormat.SORTED) {
+            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
+          }
+          readSortedField(info, meta);
+        } else {
+          throw new AssertionError();
+        }
+      } else if (type == Lucene54DocValuesFormat.SORTED_NUMERIC) {
+        SortedSetEntry ss = readSortedSetEntry(meta);
+        sortedNumerics.put(info.name, ss);
+        if (ss.format == SORTED_WITH_ADDRESSES) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          numerics.put(info.name, readNumericEntry(info, meta));
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          NumericEntry ordIndex = readNumericEntry(info, meta);
+          ordIndexes.put(info.name, ordIndex);
+        } else if (ss.format == SORTED_SET_TABLE) {
+          if (meta.readVInt() != info.number) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          NumericEntry n = readNumericEntry(info, meta);
+          ords.put(info.name, n);
+        } else if (ss.format == SORTED_SINGLE_VALUED) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
+            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
+          }
+          numerics.put(info.name, readNumericEntry(info, meta));
+        } else {
+          throw new AssertionError();
+        }
+      } else {
+        throw new CorruptIndexException("invalid type: " + type, meta);
+      }
+      fieldNumber = meta.readVInt();
+    }
+    return numFields;
+  }
+
+  private NumericEntry readNumericEntry(FieldInfo info, IndexInput meta) throws IOException {
+    NumericEntry entry = new NumericEntry();
+    entry.format = meta.readVInt();
+    entry.missingOffset = meta.readLong();
+    if (entry.format == SPARSE_COMPRESSED) {
+      // sparse bits need a bit more metadata
+      entry.numDocsWithValue = meta.readVLong();
+      final int blockShift = meta.readVInt();
+      entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithValue, blockShift);
+      ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
+      directAddressesMeta.put(info.name, entry.monotonicMeta);
+    }
+    entry.offset = meta.readLong();
+    entry.count = meta.readVLong();
+    switch(entry.format) {
+      case CONST_COMPRESSED:
+        entry.minValue = meta.readLong();
+        if (entry.count > Integer.MAX_VALUE) {
+          // currently just a limitation e.g. of bits interface and so on.
+          throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
+        }
+        break;
+      case GCD_COMPRESSED:
+        entry.minValue = meta.readLong();
+        entry.gcd = meta.readLong();
+        entry.bitsPerValue = meta.readVInt();
+        break;
+      case TABLE_COMPRESSED:
+        final int uniqueValues = meta.readVInt();
+        if (uniqueValues > 256) {
+          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
+        }
+        entry.table = new long[uniqueValues];
+        for (int i = 0; i < uniqueValues; ++i) {
+          entry.table[i] = meta.readLong();
+        }
+        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+        entry.bitsPerValue = meta.readVInt();
+        break;
+      case DELTA_COMPRESSED:
+        entry.minValue = meta.readLong();
+        entry.bitsPerValue = meta.readVInt();
+        break;
+      case MONOTONIC_COMPRESSED:
+        final int blockShift = meta.readVInt();
+        entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, maxDoc + 1, blockShift);
+        ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
+        directAddressesMeta.put(info.name, entry.monotonicMeta);
+        break;
+      case SPARSE_COMPRESSED:
+        final byte numberType = meta.readByte();
+        switch (numberType) {
+          case 0:
+            entry.numberType = NumberType.VALUE;
+            break;
+          case 1:
+            entry.numberType = NumberType.ORDINAL;
+            break;
+          default:
+            throw new CorruptIndexException("Number type can only be 0 or 1, got=" + numberType, meta);
+        }
+
+        // now read the numeric entry for non-missing values
+        final int fieldNumber = meta.readVInt();
+        if (fieldNumber != info.number) {
+          throw new CorruptIndexException("Field numbers mistmatch: " + fieldNumber + " != " + info.number, meta);
+        }
+        final int dvFormat = meta.readByte();
+        if (dvFormat != NUMERIC) {
+          throw new CorruptIndexException("Formats mistmatch: " + dvFormat + " != " + NUMERIC, meta);
+        }
+        entry.nonMissingValues = readNumericEntry(info, meta);
+        break;
+      default:
+        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
+    }
+    entry.endOffset = meta.readLong();
+    return entry;
+  }
+
+  private BinaryEntry readBinaryEntry(FieldInfo info, IndexInput meta) throws IOException {
+    BinaryEntry entry = new BinaryEntry();
+    entry.format = meta.readVInt();
+    entry.missingOffset = meta.readLong();
+    entry.minLength = meta.readVInt();
+    entry.maxLength = meta.readVInt();
+    entry.count = meta.readVLong();
+    entry.offset = meta.readLong();
+    switch(entry.format) {
+      case BINARY_FIXED_UNCOMPRESSED:
+        break;
+      case BINARY_PREFIX_COMPRESSED:
+        entry.addressesOffset = meta.readLong();
+        entry.packedIntsVersion = meta.readVInt();
+        entry.blockSize = meta.readVInt();
+        entry.reverseIndexOffset = meta.readLong();
+        break;
+      case BINARY_VARIABLE_UNCOMPRESSED:
+        entry.addressesOffset = meta.readLong();
+        final int blockShift = meta.readVInt();
+        entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.count + 1, blockShift);
+        ramBytesUsed.addAndGet(entry.addressesMeta.ramBytesUsed());
+        directAddressesMeta.put(info.name, entry.addressesMeta);
+        entry.addressesEndOffset = meta.readLong();
+        break;
+      default:
+        throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+    }
+    return entry;
+  }
+
+  SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
+    SortedSetEntry entry = new SortedSetEntry();
+    entry.format = meta.readVInt();
+    if (entry.format == SORTED_SET_TABLE) {
+      final int totalTableLength = meta.readInt();
+      if (totalTableLength > 256) {
+        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
+      }
+      entry.table = new long[totalTableLength];
+      for (int i = 0; i < totalTableLength; ++i) {
+        entry.table[i] = meta.readLong();
+      }
+      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
+      final int tableSize = meta.readInt();
+      if (tableSize > totalTableLength + 1) { // +1 because of the empty set
+        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
+      }
+      entry.tableOffsets = new int[tableSize + 1];
+      for (int i = 1; i < entry.tableOffsets.length; ++i) {
+        entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
+      }
+      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
+    } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
+      throw new CorruptIndexException("Unknown format: " + entry.format, meta);
+    }
+    return entry;
+  }
+
+  @Override
+  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+    NumericEntry entry = numerics.get(field.name);
+    Bits docsWithField;
+
+    if (entry.format == SPARSE_COMPRESSED) {
+      return getSparseNumericDocValues(entry);
+    } else {
+      if (entry.missingOffset == ALL_MISSING) {
+        return DocValues.emptyNumeric();
+      } else if (entry.missingOffset == ALL_LIVE) {
+        LongValues values = getNumeric(entry);
+        return new NumericDocValues() {
+          private int docID = -1;
+
+          @Override
+          public int docID() {
+            return docID;
+          }
+
+          @Override
+          public int nextDoc() {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+            }
+            return docID;
+          }
+
+          @Override
+          public int advance(int target) {
+            if (target >= maxDoc) {
+              docID = NO_MORE_DOCS;
+            } else {
+              docID = target;
+            }
+            return docID;
+          }
+
+          @Override
+          public long cost() {
+            // TODO
+            return 0;
+          }
+
+          @Override
+          public long longValue() {
+            return values.get(docID);
+          }
+        };
+      } else {
+        docsWithField = getLiveBits(entry.missingOffset, maxDoc);
+      }
+    }
+    return new LegacyNumericDocValuesWrapper(docsWithField, getNumeric(entry));
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed.get();
+  }
+
+  @Override
+  public synchronized Collection<Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
+    resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
+    resources.addAll(Accountables.namedAccountables("direct addresses meta field", directAddressesMeta));
+    return Collections.unmodifiableList(resources);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(data);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + numFields + ")";
+  }
+
+  LongValues getNumeric(NumericEntry entry) throws IOException {
+    switch (entry.format) {
+      case CONST_COMPRESSED: {
+        final long constant = entry.minValue;
+        final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
+        return new LongValues() {
+          @Override
+          public long get(long index) {
+            return live.get((int)index) ? constant : 0;
+          }
+        };
+      }
+      case DELTA_COMPRESSED: {
+        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+        final long delta = entry.minValue;
+        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
+        return new LongValues() {
+          @Override
+          public long get(long id) {
+            return delta + values.get(id);
+          }
+        };
+      }
+      case GCD_COMPRESSED: {
+        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+        final long min = entry.minValue;
+        final long mult = entry.gcd;
+        final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
+        return new LongValues() {
+          @Override
+          public long get(long id) {
+            return min + mult * quotientReader.get(id);
+          }
+        };
+      }
+      case TABLE_COMPRESSED: {
+        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+        final long table[] = entry.table;
+        final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
+        return new LongValues() {
+          @Override
+          public long get(long id) {
+            return table[(int) ords.get(id)];
+          }
+        };
+      }
+      case SPARSE_COMPRESSED:
+        final SparseNumericDocValues values = getSparseNumericDocValues(entry);
+        final long missingValue;
+        switch (entry.numberType) {
+          case ORDINAL:
+            missingValue = -1L;
+            break;
+          case VALUE:
+            missingValue = 0L;
+            break;
+          default:
+            throw new AssertionError();
+        }
+        return new SparseNumericDocValuesRandomAccessWrapper(values, missingValue);
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  static final class SparseNumericDocValues extends NumericDocValues {
+
+    final int docIDsLength;
+    final LongValues docIds, values;
+
+    int index, doc;
+
+    SparseNumericDocValues(int docIDsLength, LongValues docIDs, LongValues values) {
+      this.docIDsLength = docIDsLength;
+      this.docIds = docIDs;
+      this.values = values;
+      reset();
+    }
+
+    void reset() {
+      index = -1;
+      doc = -1;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (index >= docIDsLength - 1) {
+        index = docIDsLength;
+        return doc = NO_MORE_DOCS;
+      }
+      return doc = (int) docIds.get(++index);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      long loIndex = index;
+      long step = 1;
+      long hiIndex;
+      int hiDoc;
+
+      // gallop forward by exponentially growing the interval
+      // in order to find an interval so that the target doc
+      // is in ]lo, hi]. Compared to a regular binary search,
+      // this optimizes the case that the caller performs many
+      // advance calls by small deltas
+      do {
+        hiIndex = index + step;
+        if (hiIndex >= docIDsLength) {
+          hiIndex = docIDsLength;
+          hiDoc = NO_MORE_DOCS;
+          break;
+        }
+        hiDoc = (int) docIds.get(hiIndex);
+        if (hiDoc >= target) {
+          break;
+        }
+        step <<= 1;
+      } while (true);
+
+      // now binary search
+      while (loIndex + 1 < hiIndex) {
+        final long midIndex = (loIndex + 1 + hiIndex) >>> 1;
+        final int midDoc = (int) docIds.get(midIndex);
+        if (midDoc >= target) {
+          hiIndex = midIndex;
+          hiDoc = midDoc;
+        } else {
+          loIndex = midIndex;
+        }
+      }
+
+      index = (int) hiIndex;
+      return doc = hiDoc;
+    }
+
+    @Override
+    public long longValue() {
+      assert index >= 0;
+      assert index < docIDsLength;
+      return values.get(index);
+    }
+
+    @Override
+    public long cost() {
+      return docIDsLength;
+    }
+  }
+
+  static class SparseNumericDocValuesRandomAccessWrapper extends LongValues {
+
+    final SparseNumericDocValues values;
+    final long missingValue;
+
+    SparseNumericDocValuesRandomAccessWrapper(SparseNumericDocValues values, long missingValue) {
+      this.values = values;
+      this.missingValue = missingValue;
+    }
+
+    @Override
+    public long get(long longIndex) {
+      final int index = Math.toIntExact(longIndex);
+      int doc = values.docID();
+      if (doc >= index) {
+        values.reset();
+      }
+      assert values.docID() < index;
+      try {
+        doc = values.advance(index);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      if (doc == index) {
+        return values.longValue();
+      } else {
+        return missingValue;
+      }
+    }
+
+  }
+
+  LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException {
+    BinaryEntry bytes = binaries.get(field.name);
+    switch(bytes.format) {
+      case BINARY_FIXED_UNCOMPRESSED:
+        return getFixedBinary(field, bytes);
+      case BINARY_VARIABLE_UNCOMPRESSED:
+        return getVariableBinary(field, bytes);
+      case BINARY_PREFIX_COMPRESSED:
+        return getCompressedBinary(field, bytes);
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  @Override
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryEntry be = binaries.get(field.name);
+    return new LegacyBinaryDocValuesWrapper(getLiveBits(be.missingOffset, maxDoc), getLegacyBinary(field));
+  }
+
+  private LegacyBinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
+
+    final BytesRef term = new BytesRef(bytes.maxLength);
+    final byte[] buffer = term.bytes;
+    final int length = term.length = bytes.maxLength;
+
+    return new LongBinaryDocValues() {
+      @Override
+      public BytesRef get(long id) {
+        try {
+          data.seek(id * length);
+          data.readBytes(buffer, 0, buffer.length);
+          return term;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+
+  private LegacyBinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final RandomAccessInput addressesData = this.data.randomAccessSlice(bytes.addressesOffset, bytes.addressesEndOffset - bytes.addressesOffset);
+    final LongValues addresses = DirectMonotonicReader.getInstance(bytes.addressesMeta, addressesData);
+
+    final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
+    final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
+    final byte buffer[] = term.bytes;
+
+    return new LongBinaryDocValues() {
+      @Override
+      public BytesRef get(long id) {
+        long startAddress = addresses.get(id);
+        long endAddress = addresses.get(id+1);
+        int length = (int) (endAddress - startAddress);
+        try {
+          data.seek(startAddress);
+          data.readBytes(buffer, 0, length);
+          term.length = length;
+          return term;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+
+  /** returns an address instance for prefix-compressed binary values. */
+  private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
+    if (addresses == null) {
+      data.seek(bytes.addressesOffset);
+      final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
+      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      if (!merging) {
+        addressInstances.put(field.name, addresses);
+        ramBytesUsed.addAndGet(addresses.ramBytesUsed() + Integer.BYTES);
+      }
+    }
+    return addresses;
+  }
+
+  /** returns a reverse lookup instance for prefix-compressed binary values. */
+  private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
+    ReverseTermsIndex index = reverseIndexInstances.get(field.name);
+    if (index == null) {
+      index = new ReverseTermsIndex();
+      data.seek(bytes.reverseIndexOffset);
+      long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
+      index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
+      long dataSize = data.readVLong();
+      PagedBytes pagedBytes = new PagedBytes(15);
+      pagedBytes.copy(data, dataSize);
+      index.terms = pagedBytes.freeze(true);
+      if (!merging) {
+        reverseIndexInstances.put(field.name, index);
+        ramBytesUsed.addAndGet(index.ramBytesUsed());
+      }
+    }
+    return index;
+  }
+
+  private LegacyBinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
+    final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
+    assert addresses.size() > 0; // we don't have to handle empty case
+    IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
+    return new CompressedBinaryDocValues(bytes, addresses, index, slice);
+  }
+
+  @Override
+  public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    final int valueCount = (int) binaries.get(field.name).count;
+    final LegacyBinaryDocValues binary = getLegacyBinary(field);
+    NumericEntry entry = ords.get(field.name);
+    final LongValues ordinals = getNumeric(entry);
+    if (entry.format == SPARSE_COMPRESSED) {
+      final SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) ordinals).values;
+      return new SortedDocValues() {
+
+        @Override
+        public int ordValue() {
+          return (int) sparseValues.longValue();
+        }
+
+        @Override
+        public BytesRef lookupOrd(int ord) {
+          return binary.get(ord);
+        }
+
+        @Override
+        public int getValueCount() {
+          return valueCount;
+        }
+
+        @Override
+        public int docID() {
+          return sparseValues.docID();
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          return sparseValues.nextDoc();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          return sparseValues.advance(target);
+        }
+
+        @Override
+        public long cost() {
+          return sparseValues.cost();
+        }
+
+      };
+    }
+    return new SortedDocValues() {
+      private int docID = -1;
+      private int ord;
+
+      @Override
+      public int docID() {
+        return docID;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        assert docID != NO_MORE_DOCS;
+        while (true) {
+          docID++;
+          if (docID == maxDoc) {
+            docID = NO_MORE_DOCS;
+            break;
+          }
+          ord = (int) ordinals.get(docID);
+          if (ord != -1) {
+            break;
+          }
+        }
+        return docID;
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        if (target >= maxDoc) {
+          docID = NO_MORE_DOCS;
+          return docID;
+        } else {
+          docID = target-1;
+          return nextDoc();
+        }
+      }
+          
+      @Override
+      public int ordValue() {
+        return ord;
+      }
+
+      @Override
+      public long cost() {
+        // TODO
+        return 0;
+      }
+
+      @Override
+      public BytesRef lookupOrd(int ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public int getValueCount() {
+        return valueCount;
+      }
+
+      @Override
+      public int lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+    };
+  }
+
+  /** returns an address instance for sortedset ordinal lists */
+  private LongValues getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
+    RandomAccessInput data = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
+    return DirectMonotonicReader.getInstance(entry.monotonicMeta, data);
+  }
+
+  @Override
+  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+    SortedSetEntry ss = sortedNumerics.get(field.name);
+    if (ss.format == SORTED_SINGLE_VALUED) {
+      NumericEntry numericEntry = numerics.get(field.name);
+      final LongValues values = getNumeric(numericEntry);
+      if (numericEntry.format == SPARSE_COMPRESSED) {
+        SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) values).values;
+        return new SortedNumericDocValues() {
+
+          @Override
+          public long nextValue() throws IOException {
+            return sparseValues.longValue();
+          }
+
+          @Override
+          public int docValueCount() {
+            return 1;
+          }
+
+          @Override
+          public int docID() {
+            return sparseValues.docID();
+          }
+
+          @Override
+          public int nextDoc() throws IOException {
+            return sparseValues.nextDoc();
+          }
+
+          @Override
+          public int advance(int target) throws IOException {
+            return sparseValues.advance(target);
+          }
+
+          @Override
+          public long cost() {
+            return sparseValues.cost();
+          }
+
+        };
+      }
+      final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
+      return new SortedNumericDocValues() {
+        int docID = -1;
+
+        @Override
+        public int docID() {
+          return docID;
+        }
+
+        @Override
+        public int nextDoc() {
+          while (true) {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+              break;
+            }
+
+            if (docsWithField.get(docID)) {
+              // TODO: use .nextSetBit here, at least!!
+              break;
+            }
+          }
+          return docID;
+        }
+
+        @Override
+        public int advance(int target) {
+          if (target >= maxDoc) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          } else {
+            docID = target-1;
+            return nextDoc();
+          }
+        }
+
+        @Override
+        public long cost() {
+          // TODO
+          return 0;
+        }
+
+        @Override
+        public int docValueCount() {
+          return 1;
+        }
+
+        @Override
+        public long nextValue() {
+          return values.get(docID);
+        }
+      };
+    } else if (ss.format == SORTED_WITH_ADDRESSES) {
+      NumericEntry numericEntry = numerics.get(field.name);
+      final LongValues values = getNumeric(numericEntry);
+      final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+
+      return new SortedNumericDocValues() {
+        long startOffset;
+        long endOffset;
+        int docID = -1;
+        long upto;
+
+        @Override
+        public int docID() {
+          return docID;
+        }
+
+        @Override
+        public int nextDoc() {
+          while (true) {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            startOffset = ordIndex.get(docID);
+            endOffset = ordIndex.get(docID+1L);
+            if (endOffset > startOffset) {
+              break;
+            }
+          }
+          upto = startOffset;
+          return docID;
+        }
+
+        @Override
+        public int advance(int target) {
+          if (target >= maxDoc) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          } else {
+            docID = target-1;
+            return nextDoc();
+          }
+        }
+        
+        @Override
+        public long cost() {
+          // TODO
+          return 0;
+        }
+        
+        @Override
+        public int docValueCount() {
+          return (int) (endOffset - startOffset);
+        }
+        
+        @Override
+        public long nextValue() {
+          return values.get(upto++);
+        }
+      };
+    } else if (ss.format == SORTED_SET_TABLE) {
+      NumericEntry entry = ords.get(field.name);
+      final LongValues ordinals = getNumeric(entry);
+
+      final long[] table = ss.table;
+      final int[] offsets = ss.tableOffsets;
+      return new SortedNumericDocValues() {
+        int startOffset;
+        int endOffset;
+        int docID = -1;
+        int upto;
+
+        @Override
+        public int docID() {
+          return docID;
+        }
+
+        @Override
+        public int nextDoc() {
+          while (true) {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            int ord = (int) ordinals.get(docID);
+            startOffset = offsets[ord];
+            endOffset = offsets[ord+1];
+            if (endOffset > startOffset) {
+              break;
+            }
+          }
+          upto = startOffset;
+          return docID;
+        }
+
+        @Override
+        public int advance(int target) {
+          if (target >= maxDoc) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          } else {
+            docID = target-1;
+            return nextDoc();
+          }
+        }
+        
+        @Override
+        public long cost() {
+          // TODO
+          return 0;
+        }
+
+        @Override
+        public int docValueCount() {
+          return endOffset - startOffset;
+        }
+        
+        @Override
+        public long nextValue() {
+          return table[upto++];
+        }
+      };
+    } else {
+      throw new AssertionError();
+    }
+  }
+
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    SortedSetEntry ss = sortedSets.get(field.name);
+    switch (ss.format) {
+      case SORTED_SINGLE_VALUED:
+        return DocValues.singleton(getSorted(field));
+      case SORTED_WITH_ADDRESSES:
+        return getSortedSetWithAddresses(field);
+      case SORTED_SET_TABLE:
+        return getSortedSetTable(field, ss);
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
+    final long valueCount = binaries.get(field.name).count;
+    // we keep the byte[]s and list of ords on disk, these could be large
+    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
+    final LongValues ordinals = getNumeric(ords.get(field.name));
+    // but the addresses to the ord stream are in RAM
+    final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
+
+    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
+      long startOffset;
+      long offset;
+      long endOffset;
+
+      @Override
+      public long nextOrd() {
+        if (offset == endOffset) {
+          return NO_MORE_ORDS;
+        } else {
+          long ord = ordinals.get(offset);
+          offset++;
+          return ord;
+        }
+      }
+
+      @Override
+      public void setDocument(int docID) {
+        startOffset = offset = ordIndex.get(docID);
+        endOffset = ordIndex.get(docID+1L);
+      }
+
+      @Override
+      public BytesRef lookupOrd(long ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public long getValueCount() {
+        return valueCount;
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues)binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+      }, maxDoc);
+  }
+
+  private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
+    final long valueCount = binaries.get(field.name).count;
+    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
+    final NumericEntry ordinalsEntry = ords.get(field.name);
+    final LongValues ordinals = getNumeric(ordinalsEntry);
+
+    final long[] table = ss.table;
+    final int[] offsets = ss.tableOffsets;
+
+    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
+
+      int offset, startOffset, endOffset;
+
+      @Override
+      public void setDocument(int docID) {
+        final int ord = (int) ordinals.get(docID);
+        offset = startOffset = offsets[ord];
+        endOffset = offsets[ord + 1];
+      }
+
+      @Override
+      public long nextOrd() {
+        if (offset == endOffset) {
+          return NO_MORE_ORDS;
+        } else {
+          return table[offset++];
+        }
+      }
+
+      @Override
+      public BytesRef lookupOrd(long ord) {
+        return binary.get(ord);
+      }
+
+      @Override
+      public long getValueCount() {
+        return valueCount;
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues) binary).lookupTerm(key);
+        } else {
+          return super.lookupTerm(key);
+        }
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        if (binary instanceof CompressedBinaryDocValues) {
+          return ((CompressedBinaryDocValues) binary).getTermsEnum();
+        } else {
+          return super.termsEnum();
+        }
+      }
+      }, maxDoc);
+  }
+
+  private Bits getLiveBits(final long offset, final int count) throws IOException {
+    if (offset == ALL_MISSING) {
+      return new Bits.MatchNoBits(count);
+    } else if (offset == ALL_LIVE) {
+      return new Bits.MatchAllBits(count);
+    } else {
+      int length = (int) ((count + 7L) >>> 3);
+      final RandomAccessInput in = data.randomAccessSlice(offset, length);
+      return new Bits() {
+        @Override
+        public boolean get(int index) {
+          try {
+            return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+
+        @Override
+        public int length() {
+          return count;
+        }
+      };
+    }
+  }
+
+  private SparseNumericDocValues getSparseNumericDocValues(NumericEntry entry) throws IOException {
+    final RandomAccessInput docIdsData = this.data.randomAccessSlice(entry.missingOffset, entry.offset - entry.missingOffset);
+    final LongValues docIDs = DirectMonotonicReader.getInstance(entry.monotonicMeta, docIdsData);
+    final LongValues values = getNumeric(entry.nonMissingValues); // cannot be sparse
+    return new SparseNumericDocValues(Math.toIntExact(entry.numDocsWithValue), docIDs, values);
+  }
+
+  @Override
+  public synchronized DocValuesProducer getMergeInstance() throws IOException {
+    return new Lucene54DocValuesProducer(this);
+  }
+
+  @Override
+  public void close() throws IOException {
+    data.close();
+  }
+
+  /** metadata entry for a numeric docvalues field */
+  static class NumericEntry {
+    private NumericEntry() {}
+    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+    long missingOffset;
+    /** offset to the actual numeric values */
+    public long offset;
+    /** end offset to the actual numeric values */
+    public long endOffset;
+    /** bits per value used to pack the numeric values */
+    public int bitsPerValue;
+
+    int format;
+    /** count of values written */
+    public long count;
+
+    /** monotonic meta */
+    public DirectMonotonicReader.Meta monotonicMeta;
+
+    long minValue;
+    long gcd;
+    long table[];
+
+    /** for sparse compression */
+    long numDocsWithValue;
+    NumericEntry nonMissingValues;
+    NumberType numberType;
+
+  }
+
+  /** metadata entry for a binary docvalues field */
+  static class BinaryEntry {
+    private BinaryEntry() {}
+    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
+    long missingOffset;
+    /** offset to the actual binary values */
+    long offset;
+
+    int format;
+    /** count of values written */
+    public long count;
+    int minLength;
+    int maxLength;
+    /** offset to the addressing data that maps a value to its slice of the byte[] */
+    public long addressesOffset, addressesEndOffset;
+    /** meta data for addresses */
+    public DirectMonotonicReader.Meta addressesMeta;
+    /** offset to the reverse index */
+    public long reverseIndexOffset;
+    /** packed ints version used to encode addressing information */
+    public int packedIntsVersion;
+    /** packed ints blocksize */
+    public int blockSize;
+  }
+
+  /** metadata entry for a sorted-set docvalues field */
+  static class SortedSetEntry {
+    private SortedSetEntry() {}
+    int format;
+
+    long[] table;
+    int[] tableOffsets;
+  }
+
+  // internally we compose complex dv (sorted/sortedset) from other ones
+  static abstract class LongBinaryDocValues extends LegacyBinaryDocValues {
+    @Override
+    public final BytesRef get(int docID) {
+      return get((long)docID);
+    }
+
+    abstract BytesRef get(long id);
+  }
+
+  // used for reverse lookup to a small range of blocks
+  static class ReverseTermsIndex implements Accountable {
+    public MonotonicBlockPackedReader termAddresses;
+    public PagedBytes.Reader terms;
+
+    @Override
+    public long ramBytesUsed() {
+      return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
+    }
+
+    @Override
+    public Collection<Accountable> getChildResources() {
+      List<Accountable> resources = new ArrayList<>();
+      resources.add(Accountables.namedAccountable("term bytes", terms));
+      resources.add(Accountables.namedAccountable("term addresses", termAddresses));
+      return Collections.unmodifiableList(resources);
+    }
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
+    }
+  }
+
+  //in the compressed case, we add a few additional operations for
+  //more efficient reverse lookup and enumeration
+  static final class CompressedBinaryDocValues extends LongBinaryDocValues {
+    final long numValues;
+    final long numIndexValues;
+    final int maxTermLength;
+    final MonotonicBlockPackedReader addresses;
+    final IndexInput data;
+    final CompressedBinaryTermsEnum termsEnum;
+    final PagedBytes.Reader reverseTerms;
+    final MonotonicBlockPackedReader reverseAddresses;
+    final long numReverseIndexValues;
+
+    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
+      this.maxTermLength = bytes.maxLength;
+      this.numValues = bytes.count;
+      this.addresses = addresses;
+      this.numIndexValues = addresses.size();
+      this.data = data;
+      this.reverseTerms = index.terms;
+      this.reverseAddresses = index.termAddresses;
+      this.numReverseIndexValues = reverseAddresses.size();
+      this.termsEnum = getTermsEnum(data);
+    }
+
+    @Override
+    public BytesRef get(long id) {
+      try {
+        termsEnum.seekExact(id);
+        return termsEnum.term();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    long lookupTerm(BytesRef key) {
+      try {
+        switch (termsEnum.seekCeil(key)) {
+          case FOUND: return termsEnum.ord();
+          case NOT_FOUND: return -termsEnum.ord()-1;
+          default: return -numValues-1;
+        }
+      } catch (IOException bogus) {
+        throw new RuntimeException(bogus);
+      }
+    }
+
+    TermsEnum getTermsEnum() {
+      try {
+        return getTermsEnum(data.clone());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
+      return new CompressedBinaryTermsEnum(input);
+    }
+
+    class CompressedBinaryTermsEnum extends TermsEnum {
+      private long currentOrd = -1;
+      // offset to the start of the current block
+      private long currentBlockStart;
+      private final IndexInput input;
+      // delta from currentBlockStart to start of each term
+      private final int offsets[] = new int[INTERVAL_COUNT];
+      private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
+
+      private final BytesRef term = new BytesRef(maxTermLength);
+      private final BytesRef firstTerm = new BytesRef(maxTermLength);
+      private final BytesRef scratch = new BytesRef();
+
+      CompressedBinaryTermsEnum(IndexInput input) throws IOException {
+        this.input = input;
+        input.seek(0);
+      }
+
+      private void readHeader() throws IOException {
+        firstTerm.length = input.readVInt();
+        input.readBytes(firstTerm.bytes, 0, firstTerm.length);
+        input.readBytes(buffer, 0, INTERVAL_COUNT-1);
+        if (buffer[0] == -1) {
+          readShortAddresses();
+        } else {
+          readByteAddresses();
+        }
+        currentBlockStart = input.getFilePointer();
+      }
+
+      // read single byte addresses: each is delta - 2
+      // (shared prefix byte and length > 0 are both implicit)
+      private void readByteAddresses() throws IOException {
+        int addr = 0;
+        for (int i = 1; i < offsets.length; i++) {
+          addr += 2 + (buffer[i-1] & 0xFF);
+          offsets[i] = addr;
+        }
+      }
+
+      // read double byte addresses: each is delta - 2
+      // (shared prefix byte and length > 0 are both implicit)
+      private void readShortAddresses() throws IOException {
+        input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
+        int addr = 0;
+        for (int i = 1; i < offsets.length; i++) {
+          int x = i<<1;
+          addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
+          offsets[i] = addr;
+        }
+      }
+
+      // set term to the first term
+      private void readFirstTerm() throws IOException {
+        term.length = firstTerm.length;
+        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
+      }
+
+      // read term at offset, delta encoded from first term
+      private void readTerm(int offset) throws IOException {
+        int start = input.readByte() & 0xFF;
+        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
+        int suffix = offsets[offset] - offsets[offset-1] - 1;
+        input.readBytes(term.bytes, start, suffix);
+        term.length = start + suffix;
+      }
+
+      @Override
+      public BytesRef next() throws IOException {
+        currentOrd++;
+        if (currentOrd >= numValues) {
+          return null;
+        } else {
+          int offset = (int) (currentOrd & INTERVAL_MASK);
+          if (offset == 0) {
+            // switch to next block
+            readHeader();
+            readFirstTerm();
+          } else {
+            readTerm(offset);
+          }
+          return term;
+        }
+      }
+
+      // binary search reverse index to find smaller
+      // range of blocks to search
+      long binarySearchIndex(BytesRef text) throws IOException {
+        long low = 0;
+        long high = numReverseIndexValues - 1;
+        while (low <= high) {
+          long mid = (low + high) >>> 1;
+          reverseTerms.fill(scratch, reverseAddresses.get(mid));
+          int cmp = scratch.compareTo(text);
+
+          if (cmp < 0) {
+            low = mid + 1;
+          } else if (cmp > 0) {
+            high = mid - 1;
+          } else {
+            return mid;
+          }
+        }
+        return high;
+      }
+
+      // binary search against first term in block range
+      // to find term's block
+      long binarySearchBlock(BytesRef text, long low, long high) throws IOException {
+        while (low <= high) {
+          long mid = (low + high) >>> 1;
+          input.seek(addresses.get(mid));
+          term.length = input.readVInt();
+          input.readBytes(term.bytes, 0, term.length);
+          int cmp = term.compareTo(text);
+
+          if (cmp < 0) {
+            low = mid + 1;
+          } else if (cmp > 0) {
+            high = mid - 1;
+          } else {
+            return mid;
+          }
+        }
+        return high;
+      }
+
+      @Override
+      public SeekStatus seekCeil(BytesRef text) throws IOException {
+        // locate block: narrow to block range with index, then search blocks
+        final long block;
+        long indexPos = binarySearchIndex(text);
+        if (indexPos < 0) {
+          block = 0;
+        } else {
+          long low = indexPos << BLOCK_INTERVAL_SHIFT;
+          long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
+          block = Math.max(low, binarySearchBlock(text, low, high));
+        }
+
+        // position before block, then scan to term.
+        input.seek(addresses.get(block));
+        currentOrd = (block << INTERVAL_SHIFT) - 1;
+
+        while (next() != null) {
+          int cmp = term.compareTo(text);
+          if (cmp == 0) {
+            return SeekStatus.FOUND;
+          } else if (cmp > 0) {
+            return SeekStatus.NOT_FOUND;
+          }
+        }
+        return SeekStatus.END;
+      }
+
+      @Override
+      public void seekExact(long ord) throws IOException {
+        long block = ord >>> INTERVAL_SHIFT;
+        if (block != currentOrd >>> INTERVAL_SHIFT) {
+          // switch to different block
+          input.seek(addresses.get(block));
+          readHeader();
+        }
+
+        currentOrd = ord;
+
+        int offset = (int) (ord & INTERVAL_MASK);
+        if (offset == 0) {
+          readFirstTerm();
+        } else {
+          input.seek(currentBlockStart + offsets[offset-1]);
+          readTerm(offset);
+        }
+      }
+
+      @Override
+      public BytesRef term() throws IOException {
+        return term;
+      }
+
+      @Override
+      public long ord() throws IOException {
+        return currentOrd;
+      }
+
+      @Override
+      public int docFreq() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long totalTermFreq() throws IOException {
+        return -1;
+      }
+
+      @Override
+      public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
index 32c1752..ed74aa8 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
@@ -171,7 +171,7 @@ public class Lucene60Codec extends Codec {
   private final NormsFormat normsFormat = new Lucene53NormsFormat();
 
   @Override
-  public final NormsFormat normsFormat() {
+  public NormsFormat normsFormat() {
     return normsFormat;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
new file mode 100644
index 0000000..58b07eb
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene62;
+
+import java.util.Objects;
+
+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.lucene50.Lucene50CompoundFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
+import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
+import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
+import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 6.2 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.codecs.lucene60 package documentation for file format details.
+ *
+ * @lucene.experimental
+ */
+public class Lucene62Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
+  private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
+  
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    public PostingsFormat getPostingsFormatForField(String field) {
+      return Lucene62Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene62Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+  
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /** 
+   * Instantiates a new codec.
+   */
+  public Lucene62Codec() {
+    this(Mode.BEST_SPEED);
+  }
+  
+  /** 
+   * Instantiates a new codec, specifying the stored fields compression
+   * mode to use.
+   * @param mode stored fields compression mode to use for newly 
+   *             flushed/merged segments.
+   */
+  public Lucene62Codec(Mode mode) {
+    super("Lucene62");
+    this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
+  }
+  
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return storedFieldsFormat;
+  }
+  
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+  
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+  
+  @Override
+  public final 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();
+  }
+
+  /** Returns the postings format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene50".
+   *  <p>
+   *  <b>WARNING:</b> if you subclass, you are responsible for index 
+   *  backwards compatibility: future version of Lucene are only 
+   *  guaranteed to be able to read the default implementation. 
+   */
+  public PostingsFormat getPostingsFormatForField(String field) {
+    return defaultFormat;
+  }
+  
+  /** Returns the docvalues format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene54".
+   *  <p>
+   *  <b>WARNING:</b> if you subclass, you are responsible for index 
+   *  backwards compatibility: future version of Lucene are only 
+   *  guaranteed to be able to read the default implementation. 
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return defaultDVFormat;
+  }
+  
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
+
+  private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html
new file mode 100644
index 0000000..74e66d9
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/package.html
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Lucene 6.2 file format.
+</body>
+</html>


[4/6] lucene-solr:master: LUCENE-7475: Make norms sparse.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
----------------------------------------------------------------------
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 875aba5..6954d7a 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
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene60.Lucene60Codec
+org.apache.lucene.codecs.lucene62.Lucene62Codec

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
----------------------------------------------------------------------
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..26984ef 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.codecs.lucene54.Lucene54DocValuesFormat

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
new file mode 100644
index 0000000..ddb968c
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene53;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.LegacyDocValuesIterables;
+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.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+
+import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
+
+/**
+ * Writer for {@link Lucene53NormsFormat}
+ */
+class Lucene53NormsConsumer extends NormsConsumer { 
+  IndexOutput data, meta;
+  final int maxDoc;
+
+  Lucene53NormsConsumer(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 addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
+    addNormsField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, maxDoc, true));
+  }
+
+  private void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    int count = 0;
+
+    for (Number nv : values) {
+      if (nv == null) {
+        throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
+      }
+      final long v = nv.longValue();
+      minValue = Math.min(minValue, v);
+      maxValue = Math.max(maxValue, v);
+      count++;
+    }
+
+    if (count != maxDoc) {
+      throw new IllegalStateException("illegal norms data for field " + field.name + ", expected count=" + maxDoc + ", got=" + count);
+    }
+
+    if (minValue == maxValue) {
+      addConstant(minValue);
+    } else if (minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
+      addByte1(values);
+    } else if (minValue >= Short.MIN_VALUE && maxValue <= Short.MAX_VALUE) {
+      addByte2(values);
+    } else if (minValue >= Integer.MIN_VALUE && maxValue <= Integer.MAX_VALUE) {
+      addByte4(values);
+    } else {
+      addByte8(values);
+    }
+  }
+
+  private void addConstant(long constant) throws IOException {
+    meta.writeByte((byte) 0);
+    meta.writeLong(constant);
+  }
+
+  private void addByte1(Iterable<Number> values) throws IOException {
+    meta.writeByte((byte) 1);
+    meta.writeLong(data.getFilePointer());
+
+    for (Number value : values) {
+      data.writeByte(value.byteValue());
+    }
+  }
+
+  private void addByte2(Iterable<Number> values) throws IOException {
+    meta.writeByte((byte) 2);
+    meta.writeLong(data.getFilePointer());
+
+    for (Number value : values) {
+      data.writeShort(value.shortValue());
+    }
+  }
+
+  private void addByte4(Iterable<Number> values) throws IOException {
+    meta.writeByte((byte) 4);
+    meta.writeLong(data.getFilePointer());
+
+    for (Number value : values) {
+      data.writeInt(value.intValue());
+    }
+  }
+
+  private void addByte8(Iterable<Number> values) throws IOException {
+    meta.writeByte((byte) 8);
+    meta.writeLong(data.getFilePointer());
+
+    for (Number value : values) {
+      data.writeLong(value.longValue());
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java
new file mode 100644
index 0000000..86a2b6a
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/Lucene53RWNormsFormat.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene53;
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.index.SegmentWriteState;
+
+public class Lucene53RWNormsFormat extends Lucene53NormsFormat {
+
+  @Override
+  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene53NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
new file mode 100644
index 0000000..80a8eee
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene53;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene62.Lucene62RWCodec;
+import org.apache.lucene.index.BaseNormsFormatTestCase;
+
+/**
+ * Tests Lucene53NormsFormat
+ */
+public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
+  private final Codec codec = new Lucene62RWCodec();
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+
+  @Override
+  protected boolean codecSupportsSparsity() {
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
new file mode 100644
index 0000000..c6ca201
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
@@ -0,0 +1,612 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene54;
+
+
+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 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.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValues;
+import org.apache.lucene.codecs.lucene54.Lucene54DocValuesProducer.SparseNumericDocValuesRandomAccessWrapper;
+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.SeekStatus;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMFile;
+import org.apache.lucene.store.RAMInputStream;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene54DocValuesFormat
+ */
+public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene54DocValuesFormat());
+
+  @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), 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), 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), 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), 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), 1, 500);
+    }
+  }
+
+  @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, int minLength, int maxLength) 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 Lucene54DocValuesFormat();
+    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);
+      final int length = TestUtil.nextInt(random(), minLength, maxLength);
+      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(TestUtil.randomSimpleString(random(), minLength, length));
+      }
+      
+      // 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());
+      }
+    }
+  }
+
+  public void testSparseLongValues() throws IOException {
+    final int iters = atLeast(5);
+    for (int iter = 0; iter < iters; ++iter) {
+      final int numDocs = TestUtil.nextInt(random(), 0, 100);
+      final int[] docIds = new int[numDocs];
+      final long[] values = new long[numDocs];
+      final int maxDoc;
+      if (numDocs == 0) {
+        maxDoc = 1 + random().nextInt(10);
+      } else {
+        docIds[0] = random().nextInt(10);
+        for (int i = 1; i < docIds.length; ++i) {
+          docIds[i] = docIds[i - 1] + 1 + random().nextInt(100);
+        }
+        maxDoc = docIds[numDocs - 1] + 1 + random().nextInt(10);
+      }
+      for (int i = 0; i < values.length; ++i) {
+        values[i] = random().nextLong();
+      }
+      final long missingValue = random().nextLong();
+      final LongValues docIdsValues = new LongValues() {
+        @Override
+        public long get(long index) {
+          return docIds[Math.toIntExact(index)];
+        }
+      };
+      final LongValues valuesValues = new LongValues() {
+        @Override
+        public long get(long index) {
+          return values[Math.toIntExact(index)];
+        }
+      };
+      final SparseNumericDocValues sparseValues = new SparseNumericDocValues(numDocs, docIdsValues, valuesValues);
+
+      // sequential access
+      assertEquals(-1, sparseValues.docID());
+      for (int i = 0; i < docIds.length; ++i) {
+        assertEquals(docIds[i], sparseValues.nextDoc());
+      }
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc());
+
+      // advance
+      for (int i = 0; i < 2000; ++i) {
+        final int target = TestUtil.nextInt(random(), 0, maxDoc);
+        int index = Arrays.binarySearch(docIds, target);
+        if (index < 0) {
+          index = -1 - index;
+        }
+        sparseValues.reset();
+        if (index > 0) {
+          assertEquals(docIds[index - 1], sparseValues.advance(Math.toIntExact(docIds[index - 1])));
+        }
+        if (index == docIds.length) {
+          assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.advance(target));
+        } else {
+          assertEquals(docIds[index], sparseValues.advance(target));
+        }
+      }
+
+      final SparseNumericDocValuesRandomAccessWrapper raWrapper = new SparseNumericDocValuesRandomAccessWrapper(sparseValues, missingValue);
+
+      // random-access
+      for (int i = 0; i < 2000; ++i) {
+        final int docId = TestUtil.nextInt(random(), 0, maxDoc - 1);
+        final int idx = Arrays.binarySearch(docIds, docId);
+        final long value = raWrapper.get(docId);
+        if (idx >= 0) {
+          assertEquals(values[idx], value);
+        } else {
+          assertEquals(missingValue, value);
+        }
+      }
+
+      // sequential access
+      for (int docId = 0; docId < maxDoc; docId += random().nextInt(3)) {
+        final int idx = Arrays.binarySearch(docIds, docId);
+        final long value = raWrapper.get(docId);
+        if (idx >= 0) {
+          assertEquals(values[idx], value);
+        } else {
+          assertEquals(missingValue, value);
+        }
+      }
+    }
+  }
+
+  @Slow
+  public void testSortedSetAroundBlockSize() throws IOException {
+    final int frontier = 1 << Lucene54DocValuesFormat.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()));
+      RAMFile buffer = new RAMFile();
+      RAMOutputStream out = new RAMOutputStream(buffer, false);
+      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);
+        }
+      }
+      out.close();
+      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);
+      RAMInputStream in = new RAMInputStream("", buffer);
+      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 << Lucene54DocValuesFormat.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()));
+      RAMFile buffer = new RAMFile();
+      RAMOutputStream out = new RAMOutputStream(buffer, false);
+      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));
+      }
+      out.close();
+      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);
+      RAMInputStream in = new RAMInputStream("", buffer);
+      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();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java
new file mode 100644
index 0000000..fcb414d
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene62;
+
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.lucene53.Lucene53RWNormsFormat;
+import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+
+public class Lucene62RWCodec extends Lucene62Codec {
+
+  private final NormsFormat normsFormat = new Lucene53RWNormsFormat();
+
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
index c0b88cc..b01924a 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
@@ -187,7 +187,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
     };
   }
   
-  private Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
+  public Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
     final OneField field = fields.get(fieldInfo.name);
     final IndexInput in = data.clone();
     final BytesRefBuilder scratch = new BytesRefBuilder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
index faa50b7..26b00ec 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
@@ -30,7 +30,6 @@ import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Bits;
 
 /**
  * plain-text norms format.
@@ -70,7 +69,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
     
     @Override
     public NumericDocValues getNorms(FieldInfo field) throws IOException {
-      return new LegacyNumericDocValuesWrapper(new Bits.MatchAllBits(impl.maxDoc), impl.getNumericNonIterator(field));
+      return new LegacyNumericDocValuesWrapper(impl.getNumericDocsWithField(field), impl.getNumericNonIterator(field));
     }
     
     @Override
@@ -117,7 +116,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
     
     @Override
     public void addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
-      impl.addNumericField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, impl.numDocs));
+      impl.addNumericField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, impl.numDocs, false));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/LegacyDocValuesIterables.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/LegacyDocValuesIterables.java b/lucene/core/src/java/org/apache/lucene/codecs/LegacyDocValuesIterables.java
index 63f93db..74c2d80 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/LegacyDocValuesIterables.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/LegacyDocValuesIterables.java
@@ -372,7 +372,8 @@ public class LegacyDocValuesIterables {
    *
    * @deprecated Consume {@link NumericDocValues} instead. */
   @Deprecated
-  public static Iterable<Number> normsIterable(final FieldInfo field, final NormsProducer normsProducer, final int maxDoc) {
+  public static Iterable<Number> normsIterable(final FieldInfo field,
+      final NormsProducer normsProducer, final int maxDoc, boolean missingAsZero) {
 
     return new Iterable<Number>() {
 
@@ -411,9 +412,11 @@ public class LegacyDocValuesIterables {
               } catch (IOException ioe) {
                 throw new RuntimeException(ioe);
               }
-            } else {
+            } else if (missingAsZero) {
               // Unlike NumericDocValues, norms should return for missing values:
               result = 0;
+            } else {
+              result = null;
             }
             return result;
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java
index f76ac06..9170c69 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package-info.java
@@ -17,7 +17,7 @@
 
 /**
  * Components from the Lucene 5.0 index format
- * See {@link org.apache.lucene.codecs.lucene53} for an overview
+ * See {@link org.apache.lucene.codecs.lucene50} for an overview
  * of the index format.
  */
 package org.apache.lucene.codecs.lucene50;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
deleted file mode 100644
index 833500c..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsConsumer.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene53;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.LegacyDocValuesIterables;
-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.SegmentWriteState;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.IOUtils;
-
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
-
-/**
- * Writer for {@link Lucene53NormsFormat}
- */
-class Lucene53NormsConsumer extends NormsConsumer { 
-  IndexOutput data, meta;
-  final int maxDoc;
-
-  Lucene53NormsConsumer(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 addNormsField(FieldInfo field, NormsProducer normsProducer) throws IOException {
-    addNormsField(field, LegacyDocValuesIterables.normsIterable(field, normsProducer, maxDoc));
-  }
-
-  private void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
-    meta.writeVInt(field.number);
-    long minValue = Long.MAX_VALUE;
-    long maxValue = Long.MIN_VALUE;
-    int count = 0;
-
-    for (Number nv : values) {
-      if (nv == null) {
-        throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
-      }
-      final long v = nv.longValue();
-      minValue = Math.min(minValue, v);
-      maxValue = Math.max(maxValue, v);
-      count++;
-    }
-
-    if (count != maxDoc) {
-      throw new IllegalStateException("illegal norms data for field " + field.name + ", expected count=" + maxDoc + ", got=" + count);
-    }
-
-    if (minValue == maxValue) {
-      addConstant(minValue);
-    } else if (minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
-      addByte1(values);
-    } else if (minValue >= Short.MIN_VALUE && maxValue <= Short.MAX_VALUE) {
-      addByte2(values);
-    } else if (minValue >= Integer.MIN_VALUE && maxValue <= Integer.MAX_VALUE) {
-      addByte4(values);
-    } else {
-      addByte8(values);
-    }
-  }
-
-  private void addConstant(long constant) throws IOException {
-    meta.writeByte((byte) 0);
-    meta.writeLong(constant);
-  }
-
-  private void addByte1(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 1);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeByte(value.byteValue());
-    }
-  }
-
-  private void addByte2(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 2);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeShort(value.shortValue());
-    }
-  }
-
-  private void addByte4(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 4);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeInt(value.intValue());
-    }
-  }
-
-  private void addByte8(Iterable<Number> values) throws IOException {
-    meta.writeByte((byte) 8);
-    meta.writeLong(data.getFilePointer());
-
-    for (Number value : values) {
-      data.writeLong(value.longValue());
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    boolean success = false;
-    try {
-      if (meta != null) {
-        meta.writeVInt(-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;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
deleted file mode 100644
index 15cdecc..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene53;
-
-
-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 5.3 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><tt>.nvd</tt>: Norms data</li>
- *   <li><tt>.nvm</tt>: Norms metadata</li>
- * </ol>
- * <ol>
- *   <li><a name="nvm"></a>
- *   <p>The Norms metadata or .nvm file.</p>
- *   <p>For each norms field, this stores metadata, such as the offset into the 
- *      Norms data (.nvd)</p>
- *   <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- *     <li>Entry --&gt; FieldNumber,BytesPerValue, Address</li>
- *     <li>FieldNumber --&gt; {@link DataOutput#writeVInt vInt}</li>
- *     <li>BytesPerValue --&gt; {@link DataOutput#writeByte byte}</li>
- *     <li>Offset --&gt; {@link DataOutput#writeLong Int64}</li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- *   <p>FieldNumber of -1 indicates the end of metadata.</p>
- *   <p>Offset is the pointer to the start of the data in the norms data (.nvd), or the singleton value 
- *      when BytesPerValue = 0</p>
- *   <li><a name="nvd"></a>
- *   <p>The Norms data or .nvd file.</p>
- *   <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
- *   <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- *     <li>Data --&gt; {@link DataOutput#writeByte(byte) byte}<sup>MaxDoc * BytesPerValue</sup></li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- * </ol>
- * @lucene.experimental
- */
-public class Lucene53NormsFormat extends NormsFormat {
-
-  /** Sole Constructor */
-  public Lucene53NormsFormat() {}
-  
-  @Override
-  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    return new Lucene53NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
-  }
-
-  @Override
-  public NormsProducer normsProducer(SegmentReadState state) throws IOException {
-    return new Lucene53NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
-  }
-  
-  private static final String DATA_CODEC = "Lucene53NormsData";
-  private static final String DATA_EXTENSION = "nvd";
-  private static final String METADATA_CODEC = "Lucene53NormsMetadata";
-  private static final String METADATA_EXTENSION = "nvm";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
deleted file mode 100644
index a97cb5a..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
+++ /dev/null
@@ -1,230 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene53;
-
-
-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.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;
-
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_START;
-
-/**
- * Reader for {@link Lucene53NormsFormat}
- */
-class Lucene53NormsProducer extends NormsProducer {
-  // metadata maps (just file pointers and minimal stuff)
-  private final Map<Integer,NormsEntry> norms = new HashMap<>();
-  private final IndexInput data;
-  private final int maxDoc;
-
-  Lucene53NormsProducer(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);
-      }
-    }
-  }
-
-  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
-    int fieldNumber = meta.readVInt();
-    while (fieldNumber != -1) {
-      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.bytesPerValue = meta.readByte();
-      switch (entry.bytesPerValue) {
-        case 0: case 1: case 2: case 4: case 8:
-          break;
-        default:
-          throw new CorruptIndexException("Invalid bytesPerValue: " + entry.bytesPerValue + ", field: " + info.name, meta);
-      }
-      entry.offset = meta.readLong();
-      norms.put(info.number, entry);
-      fieldNumber = meta.readVInt();
-    }
-  }
-
-  @Override
-  public NumericDocValues getNorms(FieldInfo field) throws IOException {
-    final NormsEntry entry = norms.get(field.number);
-
-    if (entry.bytesPerValue == 0) {
-      final long value = entry.offset;
-      return new NormsIterator(maxDoc) {
-          @Override
-          public long longValue() {
-            return value;
-          }
-        };
-    } else {
-      RandomAccessInput slice;
-      synchronized (data) {
-        switch (entry.bytesPerValue) {
-        case 1: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readByte(docID);
-            }
-          };
-        case 2: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc * 2L);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readShort(((long)docID) << 1L);
-            }
-          };
-        case 4: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc * 4L);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readInt(((long)docID) << 2L);
-            }
-          };
-        case 8: 
-          slice = data.randomAccessSlice(entry.offset, maxDoc * 8L);
-          return new NormsIterator(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return slice.readLong(((long)docID) << 3L);
-            }
-          };
-        default:
-          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);
-  }
-
-  static class NormsEntry {
-    byte bytesPerValue;
-    long offset;
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
-  }
-
-  private static abstract class NormsIterator extends NumericDocValues {
-    private final int maxDoc;
-    protected int docID = -1;
-  
-    public NormsIterator(int maxDoc) {
-      this.maxDoc = maxDoc;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int nextDoc() {
-      docID++;
-      if (docID == maxDoc) {
-        docID = NO_MORE_DOCS;
-      }
-      return docID;
-    }
-
-    @Override
-    public int advance(int target) {
-      docID = target;
-      if (docID >= maxDoc) {
-        docID = NO_MORE_DOCS;
-      }
-      return docID;
-    }
-
-    @Override
-    public long cost() {
-      // TODO
-      return 0;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java
deleted file mode 100644
index 6a03532..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Components from the Lucene 5.3 index format
- * See {@link org.apache.lucene.codecs.lucene54} for an overview
- * of the index format.
- */
-package org.apache.lucene.codecs.lucene53;


[3/6] lucene-solr:master: LUCENE-7475: Make norms sparse.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
deleted file mode 100644
index 217cd43..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
+++ /dev/null
@@ -1,797 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene54;
-
-
-import java.io.Closeable; // javadocs
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.stream.StreamSupport;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.LegacyDocValuesIterables;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LongsRef;
-import org.apache.lucene.util.MathUtil;
-import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.packed.DirectMonotonicWriter;
-import org.apache.lucene.util.packed.DirectWriter;
-import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
-import org.apache.lucene.util.packed.PackedInts;
-
-import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
-
-/** writer for {@link Lucene54DocValuesFormat} */
-final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Closeable {
-
-  enum NumberType {
-    /** Dense ordinals */
-    ORDINAL,
-    /** Random long values */
-    VALUE;
-  }
-
-  IndexOutput data, meta;
-  final int maxDoc;
-  
-  /** expert: Creates a new writer */
-  public Lucene54DocValuesConsumer(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, Lucene54DocValuesFormat.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, Lucene54DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
-      maxDoc = state.segmentInfo.maxDoc();
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(this);
-      }
-    }
-  }
-  
-  @Override
-  public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
-  }
-
-  void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) throws IOException {
-    long count = 0;
-    long minValue = Long.MAX_VALUE;
-    long maxValue = Long.MIN_VALUE;
-    long gcd = 0;
-    long missingCount = 0;
-    long zeroCount = 0;
-    // TODO: more efficient?
-    HashSet<Long> uniqueValues = null;
-    long missingOrdCount = 0;
-    if (numberType == NumberType.VALUE) {
-      uniqueValues = new HashSet<>();
-
-      for (Number nv : values) {
-        final long v;
-        if (nv == null) {
-          v = 0;
-          missingCount++;
-          zeroCount++;
-        } else {
-          v = nv.longValue();
-          if (v == 0) {
-            zeroCount++;
-          }
-        }
-
-        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 (count != 0) { // minValue needs to be set first
-            gcd = MathUtil.gcd(gcd, v - minValue);
-          }
-        }
-
-        minValue = Math.min(minValue, v);
-        maxValue = Math.max(maxValue, v);
-
-        if (uniqueValues != null) {
-          if (uniqueValues.add(v)) {
-            if (uniqueValues.size() > 256) {
-              uniqueValues = null;
-            }
-          }
-        }
-
-        ++count;
-      }
-    } else {
-      for (Number nv : values) {
-        long v = nv.longValue();
-        if (v == -1L) {
-          missingOrdCount++;
-        }
-        minValue = Math.min(minValue, v);
-        maxValue = Math.max(maxValue, v);
-        ++count;
-      }
-    }
-    
-    final long delta = maxValue - minValue;
-    final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
-    final int tableBitsRequired = uniqueValues == null
-        ? Integer.MAX_VALUE
-        : DirectWriter.bitsRequired(uniqueValues.size() - 1);
-
-    final boolean sparse; // 1% of docs or less have a value
-    switch (numberType) {
-      case VALUE:
-        sparse = (double) missingCount / count >= 0.99;
-        break;
-      case ORDINAL:
-        sparse = (double) missingOrdCount / count >= 0.99;
-        break;
-      default:
-        throw new AssertionError();
-    }
-
-    final int format;
-    if (uniqueValues != null 
-        && count <= Integer.MAX_VALUE
-        && (uniqueValues.size() == 1
-           || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
-      // either one unique value C or two unique values: "missing" and C
-      format = CONST_COMPRESSED;
-    } else if (sparse && count >= 1024) {
-      // require at least 1024 docs to avoid flipping back and forth when doing NRT search
-      format = SPARSE_COMPRESSED;
-    } else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
-      format = TABLE_COMPRESSED;
-    } else if (gcd != 0 && gcd != 1) {
-      final long gcdDelta = (maxValue - minValue) / gcd;
-      final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
-      format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
-    } else {
-      format = DELTA_COMPRESSED;
-    }
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
-    meta.writeVInt(format);
-    if (format == SPARSE_COMPRESSED) {
-      meta.writeLong(data.getFilePointer());
-      final long numDocsWithValue;
-      switch (numberType) {
-        case VALUE:
-          numDocsWithValue = count - missingCount;
-          break;
-        case ORDINAL:
-          numDocsWithValue = count - missingOrdCount;
-          break;
-        default:
-          throw new AssertionError();
-      }
-      final long maxDoc = writeSparseMissingBitset(values, numberType, numDocsWithValue);
-      assert maxDoc == count;
-    } else if (missingCount == 0) {
-      meta.writeLong(ALL_LIVE);
-    } else if (missingCount == count) {
-      meta.writeLong(ALL_MISSING);
-    } else {
-      meta.writeLong(data.getFilePointer());
-      writeMissingBitset(values);
-    }
-    meta.writeLong(data.getFilePointer());
-    meta.writeVLong(count);
-
-    switch (format) {
-      case CONST_COMPRESSED:
-        // write the constant (nonzero value in the n=2 case, singleton value otherwise)
-        meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
-        break;
-      case GCD_COMPRESSED:
-        meta.writeLong(minValue);
-        meta.writeLong(gcd);
-        final long maxDelta = (maxValue - minValue) / gcd;
-        final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
-        meta.writeVInt(bits);
-        final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
-        for (Number nv : values) {
-          long value = nv == null ? 0 : nv.longValue();
-          quotientWriter.add((value - minValue) / gcd);
-        }
-        quotientWriter.finish();
-        break;
-      case DELTA_COMPRESSED:
-        final long minDelta = delta < 0 ? 0 : minValue;
-        meta.writeLong(minDelta);
-        meta.writeVInt(deltaBitsRequired);
-        final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
-        for (Number nv : values) {
-          long v = nv == null ? 0 : nv.longValue();
-          writer.add(v - minDelta);
-        }
-        writer.finish();
-        break;
-      case TABLE_COMPRESSED:
-        final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
-        Arrays.sort(decode);
-        final HashMap<Long,Integer> encode = new HashMap<>();
-        meta.writeVInt(decode.length);
-        for (int i = 0; i < decode.length; i++) {
-          meta.writeLong(decode[i]);
-          encode.put(decode[i], i);
-        }
-        meta.writeVInt(tableBitsRequired);
-        final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
-        for (Number nv : values) {
-          ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
-        }
-        ordsWriter.finish();
-        break;
-      case SPARSE_COMPRESSED:
-        final Iterable<Number> filteredMissingValues;
-        switch (numberType) {
-          case VALUE:
-            meta.writeByte((byte) 0);
-            filteredMissingValues = new Iterable<Number>() {
-              @Override
-              public Iterator<Number> iterator() {
-                return StreamSupport
-                    .stream(values.spliterator(), false)
-                    .filter(value -> value != null)
-                    .iterator();
-              }
-            };
-            break;
-          case ORDINAL:
-            meta.writeByte((byte) 1);
-            filteredMissingValues = new Iterable<Number>() {
-              @Override
-              public Iterator<Number> iterator() {
-                return StreamSupport
-                    .stream(values.spliterator(), false)
-                    .filter(value -> value.longValue() != -1L)
-                    .iterator();
-              }
-            };
-            break;
-          default:
-            throw new AssertionError();
-        }
-        // Write non-missing values as a numeric field
-        addNumericField(field, filteredMissingValues, numberType);
-        break;
-      default:
-        throw new AssertionError();
-    }
-    meta.writeLong(data.getFilePointer());
-  }
-  
-  // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
-  // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
-  void writeMissingBitset(Iterable<?> values) throws IOException {
-    byte bits = 0;
-    int count = 0;
-    for (Object v : values) {
-      if (count == 8) {
-        data.writeByte(bits);
-        count = 0;
-        bits = 0;
-      }
-      if (v != null) {
-        bits |= 1 << (count & 7);
-      }
-      count++;
-    }
-    if (count > 0) {
-      data.writeByte(bits);
-    }
-  }
-
-  long writeSparseMissingBitset(Iterable<Number> values, NumberType numberType, long numDocsWithValue) throws IOException {
-    meta.writeVLong(numDocsWithValue);
-
-    // Write doc IDs that have a value
-    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-    final DirectMonotonicWriter docIdsWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithValue, DIRECT_MONOTONIC_BLOCK_SHIFT);
-    long docID = 0;
-    for (Number nv : values) {
-      switch (numberType) {
-        case VALUE:
-          if (nv != null) {
-            docIdsWriter.add(docID);
-          }
-          break;
-        case ORDINAL:
-          if (nv.longValue() != -1L) {
-            docIdsWriter.add(docID);
-          }
-          break;
-        default:
-          throw new AssertionError();
-      }
-      docID++;
-    }
-    docIdsWriter.finish();
-    return docID;
-  }
-
-  @Override
-  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
-  }
-  
-  private void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
-    // write the byte[] data
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.BINARY);
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = Integer.MIN_VALUE;
-    final long startFP = data.getFilePointer();
-    long count = 0;
-    long missingCount = 0;
-    for(BytesRef v : values) {
-      final int length;
-      if (v == null) {
-        length = 0;
-        missingCount++;
-      } else {
-        length = v.length;
-      }
-      minLength = Math.min(minLength, length);
-      maxLength = Math.max(maxLength, length);
-      if (v != null) {
-        data.writeBytes(v.bytes, v.offset, v.length);
-      }
-      count++;
-    }
-    meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
-    if (missingCount == 0) {
-      meta.writeLong(ALL_LIVE);
-    } else if (missingCount == count) {
-      meta.writeLong(ALL_MISSING);
-    } else {
-      meta.writeLong(data.getFilePointer());
-      writeMissingBitset(values);
-    }
-    meta.writeVInt(minLength);
-    meta.writeVInt(maxLength);
-    meta.writeVLong(count);
-    meta.writeLong(startFP);
-    
-    // if minLength == maxLength, it's a fixed-length byte[], we are done (the addresses are implicit)
-    // otherwise, we need to record the length fields...
-    if (minLength != maxLength) {
-      meta.writeLong(data.getFilePointer());
-      meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-
-      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
-      long addr = 0;
-      writer.add(addr);
-      for (BytesRef v : values) {
-        if (v != null) {
-          addr += v.length;
-        }
-        writer.add(addr);
-      }
-      writer.finish();
-      meta.writeLong(data.getFilePointer());
-    }
-  }
-  
-  /** expert: writes a value dictionary for a sorted/sortedset field */
-  private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
-    // first check if it's a "fixed-length" terms dict, and compressibility if so
-    int minLength = Integer.MAX_VALUE;
-    int maxLength = Integer.MIN_VALUE;
-    long numValues = 0;
-    BytesRefBuilder previousValue = new BytesRefBuilder();
-    long prefixSum = 0; // only valid for fixed-width data, as we have a choice there
-    for (BytesRef v : values) {
-      minLength = Math.min(minLength, v.length);
-      maxLength = Math.max(maxLength, v.length);
-      if (minLength == maxLength) {
-        int termPosition = (int) (numValues & INTERVAL_MASK);
-        if (termPosition == 0) {
-          // first term in block, save it away to compare against the last term later
-          previousValue.copyBytes(v);
-        } else if (termPosition == INTERVAL_COUNT - 1) {
-          // last term in block, accumulate shared prefix against first term
-          prefixSum += StringHelper.bytesDifference(previousValue.get(), v);
-        }
-      }
-      numValues++;
-    }
-    // for fixed width data, look at the avg(shared prefix) before deciding how to encode:
-    // prefix compression "costs" worst case 2 bytes per term because we must store suffix lengths.
-    // so if we share at least 3 bytes on average, always compress.
-    if (minLength == maxLength && prefixSum <= 3*(numValues >> INTERVAL_SHIFT)) {
-      // no index needed: not very compressible, direct addressing by mult
-      addBinaryField(field, values);
-    } else if (numValues < REVERSE_INTERVAL_COUNT) {
-      // low cardinality: waste a few KB of ram, but can't really use fancy index etc
-      addBinaryField(field, values);
-    } else {
-      assert numValues > 0; // we don't have to handle the empty case
-      // header
-      meta.writeVInt(field.number);
-      meta.writeByte(Lucene54DocValuesFormat.BINARY);
-      meta.writeVInt(BINARY_PREFIX_COMPRESSED);
-      meta.writeLong(-1L);
-      // now write the bytes: sharing prefixes within a block
-      final long startFP = data.getFilePointer();
-      // currently, we have to store the delta from expected for every 1/nth term
-      // we could avoid this, but it's not much and less overall RAM than the previous approach!
-      RAMOutputStream addressBuffer = new RAMOutputStream();
-      MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
-      // buffers up 16 terms
-      RAMOutputStream bytesBuffer = new RAMOutputStream();
-      // buffers up block header
-      RAMOutputStream headerBuffer = new RAMOutputStream();
-      BytesRefBuilder lastTerm = new BytesRefBuilder();
-      lastTerm.grow(maxLength);
-      long count = 0;
-      int suffixDeltas[] = new int[INTERVAL_COUNT];
-      for (BytesRef v : values) {
-        int termPosition = (int) (count & INTERVAL_MASK);
-        if (termPosition == 0) {
-          termAddresses.add(data.getFilePointer() - startFP);
-          // abs-encode first term
-          headerBuffer.writeVInt(v.length);
-          headerBuffer.writeBytes(v.bytes, v.offset, v.length);
-          lastTerm.copyBytes(v);
-        } else {
-          // prefix-code: we only share at most 255 characters, to encode the length as a single
-          // byte and have random access. Larger terms just get less compression.
-          int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
-          bytesBuffer.writeByte((byte) sharedPrefix);
-          bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
-          // we can encode one smaller, because terms are unique.
-          suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
-        }
-        
-        count++;
-        // flush block
-        if ((count & INTERVAL_MASK) == 0) {
-          flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
-        }
-      }
-      // flush trailing crap
-      int leftover = (int) (count & INTERVAL_MASK);
-      if (leftover > 0) {
-        Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
-        flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
-      }
-      final long indexStartFP = data.getFilePointer();
-      // write addresses of indexed terms
-      termAddresses.finish();
-      addressBuffer.writeTo(data);
-      addressBuffer = null;
-      termAddresses = null;
-      meta.writeVInt(minLength);
-      meta.writeVInt(maxLength);
-      meta.writeVLong(count);
-      meta.writeLong(startFP);
-      meta.writeLong(indexStartFP);
-      meta.writeVInt(PackedInts.VERSION_CURRENT);
-      meta.writeVInt(MONOTONIC_BLOCK_SIZE);
-      addReverseTermIndex(field, values, maxLength);
-    }
-  }
-  // writes term dictionary "block"
-  // first term is absolute encoded as vint length + bytes.
-  // lengths of subsequent N terms are encoded as either N bytes or N shorts.
-  // in the double-byte case, the first byte is indicated with -1.
-  // subsequent terms are encoded as byte suffixLength + bytes.
-  private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
-    boolean twoByte = false;
-    for (int i = 1; i < suffixDeltas.length; i++) {
-      if (suffixDeltas[i] > 254) {
-        twoByte = true;
-      }
-    }
-    if (twoByte) {
-      headerBuffer.writeByte((byte)255);
-      for (int i = 1; i < suffixDeltas.length; i++) {
-        headerBuffer.writeShort((short) suffixDeltas[i]);
-      }
-    } else {
-      for (int i = 1; i < suffixDeltas.length; i++) {
-        headerBuffer.writeByte((byte) suffixDeltas[i]);
-      }
-    }
-    headerBuffer.writeTo(data);
-    headerBuffer.reset();
-    bytesBuffer.writeTo(data);
-    bytesBuffer.reset();
-  }
-  
-  // writes reverse term index: used for binary searching a term into a range of 64 blocks
-  // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
-  // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
-  private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
-    long count = 0;
-    BytesRefBuilder priorTerm = new BytesRefBuilder();
-    priorTerm.grow(maxLength);
-    BytesRef indexTerm = new BytesRef();
-    long startFP = data.getFilePointer();
-    PagedBytes pagedBytes = new PagedBytes(15);
-    MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
-    
-    for (BytesRef b : values) {
-      int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
-      if (termPosition == 0) {
-        int len = StringHelper.sortKeyLength(priorTerm.get(), b);
-        indexTerm.bytes = b.bytes;
-        indexTerm.offset = b.offset;
-        indexTerm.length = len;
-        addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
-      } else if (termPosition == REVERSE_INTERVAL_MASK) {
-        priorTerm.copyBytes(b);
-      }
-      count++;
-    }
-    addresses.finish();
-    long numBytes = pagedBytes.getPointer();
-    pagedBytes.freeze(true);
-    PagedBytesDataInput in = pagedBytes.getDataInput();
-    meta.writeLong(startFP);
-    data.writeVLong(numBytes);
-    data.copyBytes(in, numBytes);
-  }
-
-  @Override
-  public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.SORTED);
-    addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)));
-    addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL);
-  }
-
-  private void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> ords) throws IOException {
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.SORTED);
-    addTermsDict(field, values);
-    addNumericField(field, ords, NumberType.ORDINAL);
-  }
-
-  @Override
-  public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
-
-    final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
-    final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
-    
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.SORTED_NUMERIC);
-    if (isSingleValued(docToValueCount)) {
-      meta.writeVInt(SORTED_SINGLE_VALUED);
-      // The field is single-valued, we can encode it as NUMERIC
-      addNumericField(field, singletonView(docToValueCount, values, null), NumberType.VALUE);
-    } else {
-      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
-      if (uniqueValueSets != null) {
-        meta.writeVInt(SORTED_SET_TABLE);
-
-        // write the set_id -> values mapping
-        writeDictionary(uniqueValueSets);
-
-        // write the doc -> set_id as a numeric field
-        addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), NumberType.ORDINAL);
-      } else {
-        meta.writeVInt(SORTED_WITH_ADDRESSES);
-        // write the stream of values as a numeric field
-        addNumericField(field, values, NumberType.VALUE);
-        // write the doc -> ord count as a absolute index to the stream
-        addOrdIndex(field, docToValueCount);
-      }
-    }
-  }
-
-  @Override
-  public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-
-    Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
-    Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
-    Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
-
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.SORTED_SET);
-
-    if (isSingleValued(docToOrdCount)) {
-      meta.writeVInt(SORTED_SINGLE_VALUED);
-      // The field is single-valued, we can encode it as SORTED
-      addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
-    } else {
-      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
-      if (uniqueValueSets != null) {
-        meta.writeVInt(SORTED_SET_TABLE);
-
-        // write the set_id -> ords mapping
-        writeDictionary(uniqueValueSets);
-
-        // write the ord -> byte[] as a binary field
-        addTermsDict(field, values);
-
-        // write the doc -> set_id as a numeric field
-        addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), NumberType.ORDINAL);
-      } else {
-        meta.writeVInt(SORTED_WITH_ADDRESSES);
-
-        // write the ord -> byte[] as a binary field
-        addTermsDict(field, values);
-
-        // write the stream of ords as a numeric field
-        // NOTE: we could return an iterator that delta-encodes these within a doc
-        addNumericField(field, ords, NumberType.ORDINAL);
-
-        // write the doc -> ord count as a absolute index to the stream
-        addOrdIndex(field, docToOrdCount);
-      }
-    }
-  }
-
-  private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
-    Set<LongsRef> uniqueValueSet = new HashSet<>();
-    LongsRef docValues = new LongsRef(256);
-
-    Iterator<Number> valueCountIterator = docToValueCount.iterator();
-    Iterator<Number> valueIterator = values.iterator();
-    int totalDictSize = 0;
-    while (valueCountIterator.hasNext()) {
-      docValues.length = valueCountIterator.next().intValue();
-      if (docValues.length > 256) {
-        return null;
-      }
-      for (int i = 0; i < docValues.length; ++i) {
-        docValues.longs[i] = valueIterator.next().longValue();
-      }
-      if (uniqueValueSet.contains(docValues)) {
-        continue;
-      }
-      totalDictSize += docValues.length;
-      if (totalDictSize > 256) {
-        return null;
-      }
-      uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
-    }
-    assert valueIterator.hasNext() == false;
-    return new TreeSet<>(uniqueValueSet);
-  }
-
-  private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
-    int lengthSum = 0;
-    for (LongsRef longs : uniqueValueSets) {
-      lengthSum += longs.length;
-    }
-
-    meta.writeInt(lengthSum);
-    for (LongsRef valueSet : uniqueValueSets) {
-      for (int  i = 0; i < valueSet.length; ++i) {
-        meta.writeLong(valueSet.longs[valueSet.offset + i]);
-      }
-    }
-
-    meta.writeInt(uniqueValueSets.size());
-    for (LongsRef valueSet : uniqueValueSets) {
-      meta.writeInt(valueSet.length);
-    }
-  }
-
-  private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
-    final Map<LongsRef, Integer> setIds = new HashMap<>();
-    int i = 0;
-    for (LongsRef set : uniqueValueSets) {
-      setIds.put(set, i++);
-    }
-    assert i == uniqueValueSets.size();
-
-    return new Iterable<Number>() {
-
-      @Override
-      public Iterator<Number> iterator() {
-        final Iterator<Number> valueCountIterator = docToValueCount.iterator();
-        final Iterator<Number> valueIterator = values.iterator();
-        final LongsRef docValues = new LongsRef(256);
-        return new Iterator<Number>() {
-
-          @Override
-          public boolean hasNext() {
-            return valueCountIterator.hasNext();
-          }
-
-          @Override
-          public Number next() {
-            docValues.length = valueCountIterator.next().intValue();
-            for (int i = 0; i < docValues.length; ++i) {
-              docValues.longs[i] = valueIterator.next().longValue();
-            }
-            final Integer id = setIds.get(docValues);
-            assert id != null;
-            return id;
-          }
-
-        };
-
-      }
-    };
-  }
-
-  // writes addressing information as MONOTONIC_COMPRESSED integer
-  private void addOrdIndex(FieldInfo field, Iterable<Number> values) throws IOException {
-    meta.writeVInt(field.number);
-    meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
-    meta.writeVInt(MONOTONIC_COMPRESSED);
-    meta.writeLong(-1L);
-    meta.writeLong(data.getFilePointer());
-    meta.writeVLong(maxDoc);
-    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-
-    final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
-    long addr = 0;
-    writer.add(addr);
-    for (Number v : values) {
-      addr += v.longValue();
-      writer.add(addr);
-    }
-    writer.finish();
-    meta.writeLong(data.getFilePointer());
-  }
-
-  @Override
-  public void close() throws IOException {
-    boolean success = false;
-    try {
-      if (meta != null) {
-        meta.writeVInt(-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;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
deleted file mode 100644
index 91ccfe2..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene54;
-
-
-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.codecs.lucene70.Lucene70DocValuesFormat;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.packed.DirectWriter;
-
-/**
- * Lucene 5.4 DocValues format.
- * <p>
- * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these 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 non-missing value, only the missing
- *        bitset is encoded.
- *    <li>Sparse-compressed: only documents with a value are stored, and lookups are performed
- *        using binary search.
- * </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 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 table: when there are few unique sets of values (&lt; 256) then each set is assigned
- *        an id, a lookup table is written and the mapping from document to set id is written using the
- *        numeric strategies above.
- *    <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>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
- *        an id, a lookup table is written and the mapping from document to set id is written using the
- *        numeric strategies above.
- *    <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
- *        strategies above.
- * </ul>
- * <p>
- * Files:
- * <ol>
- *   <li><tt>.dvd</tt>: DocValues data</li>
- *   <li><tt>.dvm</tt>: DocValues metadata</li>
- * </ol>
- * @lucene.experimental
- * @deprecated Use {@link Lucene70DocValuesFormat}.
- */
-@Deprecated
-public final class Lucene54DocValuesFormat extends DocValuesFormat {
-
-  /** Sole Constructor */
-  public Lucene54DocValuesFormat() {
-    super("Lucene54");
-  }
-
-  @Override
-  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new Lucene54DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
-  }
-
-  @Override
-  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new Lucene54DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
-  }
-  
-  static final String DATA_CODEC = "Lucene54DocValuesData";
-  static final String DATA_EXTENSION = "dvd";
-  static final String META_CODEC = "Lucene54DocValuesMetadata";
-  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;
-  
-  // address terms in blocks of 16 terms
-  static final int INTERVAL_SHIFT = 4;
-  static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
-  static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
-  
-  // build reverse index from every 1024th term
-  static final int REVERSE_INTERVAL_SHIFT = 10;
-  static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
-  static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
-  
-  // for conversion from reverse index to block
-  static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
-  static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
-  static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
-
-  /** Compressed using packed blocks of ints. */
-  static final int DELTA_COMPRESSED = 0;
-  /** Compressed by computing the GCD. */
-  static final int GCD_COMPRESSED = 1;
-  /** Compressed by giving IDs to unique values. */
-  static final int TABLE_COMPRESSED = 2;
-  /** Compressed with monotonically increasing values */
-  static final int MONOTONIC_COMPRESSED = 3;
-  /** Compressed with constant value (uses only missing bitset) */
-  static final int CONST_COMPRESSED = 4;
-  /** Compressed with sparse arrays. */
-  static final int SPARSE_COMPRESSED = 5;
-
-  /** Uncompressed binary, written directly (fixed length). */
-  static final int BINARY_FIXED_UNCOMPRESSED = 0;
-  /** Uncompressed binary, written directly (variable length). */
-  static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
-  /** Compressed binary with shared prefixes */
-  static final int BINARY_PREFIX_COMPRESSED = 2;
-
-  /** Standard storage for sorted set values with 1 level of indirection:
-   *  {@code docId -> address -> ord}. */
-  static final int SORTED_WITH_ADDRESSES = 0;
-  /** Single-valued sorted set values, encoded as sorted values, so no level
-   *  of indirection: {@code docId -> ord}. */
-  static final int SORTED_SINGLE_VALUED = 1;
-  /** Compressed giving IDs to unique sets of values:
-   * {@code docId -> setId -> ords} */
-  static final int SORTED_SET_TABLE = 2;
-  
-  /** placeholder for missing offset that means there are no missing values */
-  static final int ALL_LIVE = -1;
-  /** placeholder for missing offset that means all values are missing */
-  static final int ALL_MISSING = -2;
-  
-  // addressing uses 16k blocks
-  static final int MONOTONIC_BLOCK_SIZE = 16384;
-  static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
-}


[6/6] lucene-solr:master: LUCENE-7475: Make norms sparse.

Posted by jp...@apache.org.
LUCENE-7475: Make norms sparse.


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

Branch: refs/heads/master
Commit: 9128bdbaf547429667740cdc95370c7c606f83fc
Parents: 28d187a
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed Oct 5 14:07:15 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Oct 6 14:07:27 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |    3 +
 .../codecs/lucene53/Lucene53NormsFormat.java    |   91 +
 .../codecs/lucene53/Lucene53NormsProducer.java  |  230 +++
 .../lucene/codecs/lucene53/package-info.java    |   23 +
 .../lucene54/Lucene54DocValuesConsumer.java     |  797 ++++++++
 .../lucene54/Lucene54DocValuesFormat.java       |  186 ++
 .../lucene54/Lucene54DocValuesProducer.java     | 1707 ++++++++++++++++++
 .../lucene/codecs/lucene54/package-info.java    |  403 +++++
 .../lucene/codecs/lucene60/Lucene60Codec.java   |    2 +-
 .../lucene/codecs/lucene62/Lucene62Codec.java   |  176 ++
 .../apache/lucene/codecs/lucene62/package.html  |   25 +
 .../services/org.apache.lucene.codecs.Codec     |    1 +
 .../org.apache.lucene.codecs.DocValuesFormat    |    1 +
 .../codecs/lucene53/Lucene53NormsConsumer.java  |  159 ++
 .../codecs/lucene53/Lucene53RWNormsFormat.java  |   31 +
 .../lucene53/TestLucene53NormsFormat.java       |   38 +
 .../lucene54/TestLucene54DocValuesFormat.java   |  612 +++++++
 .../lucene/codecs/lucene62/Lucene62RWCodec.java |   32 +
 .../simpletext/SimpleTextDocValuesReader.java   |    2 +-
 .../simpletext/SimpleTextNormsFormat.java       |    5 +-
 .../lucene/codecs/LegacyDocValuesIterables.java |    7 +-
 .../lucene/codecs/lucene50/package-info.java    |    2 +-
 .../codecs/lucene53/Lucene53NormsConsumer.java  |  159 --
 .../codecs/lucene53/Lucene53NormsFormat.java    |   91 -
 .../codecs/lucene53/Lucene53NormsProducer.java  |  230 ---
 .../lucene/codecs/lucene53/package-info.java    |   23 -
 .../lucene54/Lucene54DocValuesConsumer.java     |  797 --------
 .../lucene54/Lucene54DocValuesFormat.java       |  186 --
 .../lucene54/Lucene54DocValuesProducer.java     | 1707 ------------------
 .../lucene/codecs/lucene54/package-info.java    |  403 -----
 .../lucene/codecs/lucene62/Lucene62Codec.java   |  176 --
 .../lucene/codecs/lucene70/Lucene70Codec.java   |    3 +-
 .../codecs/lucene70/Lucene70NormsConsumer.java  |  155 ++
 .../codecs/lucene70/Lucene70NormsFormat.java    |   97 +
 .../codecs/lucene70/Lucene70NormsProducer.java  |  271 +++
 .../lucene/codecs/lucene70/SparseDISI.java      |  115 ++
 .../lucene/codecs/lucene70/package-info.java    |    6 +-
 .../lucene/index/DefaultIndexingChain.java      |   13 +-
 .../apache/lucene/index/NormValuesWriter.java   |   53 +-
 .../services/org.apache.lucene.codecs.Codec     |    1 -
 .../org.apache.lucene.codecs.DocValuesFormat    |    1 -
 .../lucene53/TestLucene53NormsFormat.java       |   34 -
 .../lucene54/TestLucene54DocValuesFormat.java   |  612 -------
 .../lucene70/TestLucene70NormsFormat.java       |   34 +
 .../lucene/codecs/lucene70/TestSparseDISI.java  |   94 +
 .../test/org/apache/lucene/index/TestNorms.java |   21 +
 .../lucene/index/BaseNormsFormatTestCase.java   |  313 +++-
 47 files changed, 5620 insertions(+), 4508 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4437792..0a65d20 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -22,6 +22,9 @@ API Changes
   to iterators, enabling future codec compression improvements. (Mike
   McCandless)
 
+* LUCENE-7475: Norms now support sparsity, allowing to pay for what is
+  actually used. (Adrien Grand)
+
 Bug Fixes
 
 Improvements

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
new file mode 100644
index 0000000..1f7928f
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsFormat.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene53;
+
+
+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 5.3 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><tt>.nvd</tt>: Norms data</li>
+ *   <li><tt>.nvm</tt>: Norms metadata</li>
+ * </ol>
+ * <ol>
+ *   <li><a name="nvm"></a>
+ *   <p>The Norms metadata or .nvm file.</p>
+ *   <p>For each norms field, this stores metadata, such as the offset into the 
+ *      Norms data (.nvd)</p>
+ *   <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
+ *     <li>Entry --&gt; FieldNumber,BytesPerValue, Address</li>
+ *     <li>FieldNumber --&gt; {@link DataOutput#writeVInt vInt}</li>
+ *     <li>BytesPerValue --&gt; {@link DataOutput#writeByte byte}</li>
+ *     <li>Offset --&gt; {@link DataOutput#writeLong Int64}</li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </ul>
+ *   <p>FieldNumber of -1 indicates the end of metadata.</p>
+ *   <p>Offset is the pointer to the start of the data in the norms data (.nvd), or the singleton value 
+ *      when BytesPerValue = 0</p>
+ *   <li><a name="nvd"></a>
+ *   <p>The Norms data or .nvd file.</p>
+ *   <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
+ *   <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer</p>
+ *   <ul>
+ *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
+ *     <li>Data --&gt; {@link DataOutput#writeByte(byte) byte}<sup>MaxDoc * BytesPerValue</sup></li>
+ *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ *   </ul>
+ * </ol>
+ * @lucene.experimental
+ */
+public class Lucene53NormsFormat extends NormsFormat {
+
+  /** Sole Constructor */
+  public Lucene53NormsFormat() {}
+  
+  @Override
+  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
+    throw new UnsupportedOperationException("This format can only be used for reading");
+  }
+
+  @Override
+  public NormsProducer normsProducer(SegmentReadState state) throws IOException {
+    return new Lucene53NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "Lucene53NormsData";
+  static final String DATA_EXTENSION = "nvd";
+  static final String METADATA_CODEC = "Lucene53NormsMetadata";
+  static final String METADATA_EXTENSION = "nvm";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
new file mode 100644
index 0000000..a97cb5a
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene53;
+
+
+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.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;
+
+import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
+import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_START;
+
+/**
+ * Reader for {@link Lucene53NormsFormat}
+ */
+class Lucene53NormsProducer extends NormsProducer {
+  // metadata maps (just file pointers and minimal stuff)
+  private final Map<Integer,NormsEntry> norms = new HashMap<>();
+  private final IndexInput data;
+  private final int maxDoc;
+
+  Lucene53NormsProducer(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);
+      }
+    }
+  }
+
+  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int fieldNumber = meta.readVInt();
+    while (fieldNumber != -1) {
+      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.bytesPerValue = meta.readByte();
+      switch (entry.bytesPerValue) {
+        case 0: case 1: case 2: case 4: case 8:
+          break;
+        default:
+          throw new CorruptIndexException("Invalid bytesPerValue: " + entry.bytesPerValue + ", field: " + info.name, meta);
+      }
+      entry.offset = meta.readLong();
+      norms.put(info.number, entry);
+      fieldNumber = meta.readVInt();
+    }
+  }
+
+  @Override
+  public NumericDocValues getNorms(FieldInfo field) throws IOException {
+    final NormsEntry entry = norms.get(field.number);
+
+    if (entry.bytesPerValue == 0) {
+      final long value = entry.offset;
+      return new NormsIterator(maxDoc) {
+          @Override
+          public long longValue() {
+            return value;
+          }
+        };
+    } else {
+      RandomAccessInput slice;
+      synchronized (data) {
+        switch (entry.bytesPerValue) {
+        case 1: 
+          slice = data.randomAccessSlice(entry.offset, maxDoc);
+          return new NormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readByte(docID);
+            }
+          };
+        case 2: 
+          slice = data.randomAccessSlice(entry.offset, maxDoc * 2L);
+          return new NormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readShort(((long)docID) << 1L);
+            }
+          };
+        case 4: 
+          slice = data.randomAccessSlice(entry.offset, maxDoc * 4L);
+          return new NormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readInt(((long)docID) << 2L);
+            }
+          };
+        case 8: 
+          slice = data.randomAccessSlice(entry.offset, maxDoc * 8L);
+          return new NormsIterator(maxDoc) {
+            @Override
+            public long longValue() throws IOException {
+              return slice.readLong(((long)docID) << 3L);
+            }
+          };
+        default:
+          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);
+  }
+
+  static class NormsEntry {
+    byte bytesPerValue;
+    long offset;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(fields=" + norms.size() + ")";
+  }
+
+  private static abstract class NormsIterator extends NumericDocValues {
+    private final int maxDoc;
+    protected int docID = -1;
+  
+    public NormsIterator(int maxDoc) {
+      this.maxDoc = maxDoc;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      docID++;
+      if (docID == maxDoc) {
+        docID = NO_MORE_DOCS;
+      }
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      docID = target;
+      if (docID >= maxDoc) {
+        docID = NO_MORE_DOCS;
+      }
+      return docID;
+    }
+
+    @Override
+    public long cost() {
+      // TODO
+      return 0;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java
new file mode 100644
index 0000000..93fefb8
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene53/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Components from the Lucene 5.3 index format
+ * See {@link org.apache.lucene.codecs.lucene53} for an overview
+ * of the index format.
+ */
+package org.apache.lucene.codecs.lucene53;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
new file mode 100644
index 0000000..217cd43
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
@@ -0,0 +1,797 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene54;
+
+
+import java.io.Closeable; // javadocs
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.stream.StreamSupport;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.LegacyDocValuesIterables;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+import org.apache.lucene.util.packed.DirectWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
+
+/** writer for {@link Lucene54DocValuesFormat} */
+final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Closeable {
+
+  enum NumberType {
+    /** Dense ordinals */
+    ORDINAL,
+    /** Random long values */
+    VALUE;
+  }
+
+  IndexOutput data, meta;
+  final int maxDoc;
+  
+  /** expert: Creates a new writer */
+  public Lucene54DocValuesConsumer(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, Lucene54DocValuesFormat.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, Lucene54DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      maxDoc = state.segmentInfo.maxDoc();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+  
+  @Override
+  public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
+  }
+
+  void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) throws IOException {
+    long count = 0;
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    long missingCount = 0;
+    long zeroCount = 0;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    long missingOrdCount = 0;
+    if (numberType == NumberType.VALUE) {
+      uniqueValues = new HashSet<>();
+
+      for (Number nv : values) {
+        final long v;
+        if (nv == null) {
+          v = 0;
+          missingCount++;
+          zeroCount++;
+        } else {
+          v = nv.longValue();
+          if (v == 0) {
+            zeroCount++;
+          }
+        }
+
+        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 (count != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minValue);
+          }
+        }
+
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+
+        if (uniqueValues != null) {
+          if (uniqueValues.add(v)) {
+            if (uniqueValues.size() > 256) {
+              uniqueValues = null;
+            }
+          }
+        }
+
+        ++count;
+      }
+    } else {
+      for (Number nv : values) {
+        long v = nv.longValue();
+        if (v == -1L) {
+          missingOrdCount++;
+        }
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+        ++count;
+      }
+    }
+    
+    final long delta = maxValue - minValue;
+    final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
+    final int tableBitsRequired = uniqueValues == null
+        ? Integer.MAX_VALUE
+        : DirectWriter.bitsRequired(uniqueValues.size() - 1);
+
+    final boolean sparse; // 1% of docs or less have a value
+    switch (numberType) {
+      case VALUE:
+        sparse = (double) missingCount / count >= 0.99;
+        break;
+      case ORDINAL:
+        sparse = (double) missingOrdCount / count >= 0.99;
+        break;
+      default:
+        throw new AssertionError();
+    }
+
+    final int format;
+    if (uniqueValues != null 
+        && count <= Integer.MAX_VALUE
+        && (uniqueValues.size() == 1
+           || (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
+      // either one unique value C or two unique values: "missing" and C
+      format = CONST_COMPRESSED;
+    } else if (sparse && count >= 1024) {
+      // require at least 1024 docs to avoid flipping back and forth when doing NRT search
+      format = SPARSE_COMPRESSED;
+    } else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
+      format = TABLE_COMPRESSED;
+    } else if (gcd != 0 && gcd != 1) {
+      final long gcdDelta = (maxValue - minValue) / gcd;
+      final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
+      format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
+    } else {
+      format = DELTA_COMPRESSED;
+    }
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
+    meta.writeVInt(format);
+    if (format == SPARSE_COMPRESSED) {
+      meta.writeLong(data.getFilePointer());
+      final long numDocsWithValue;
+      switch (numberType) {
+        case VALUE:
+          numDocsWithValue = count - missingCount;
+          break;
+        case ORDINAL:
+          numDocsWithValue = count - missingOrdCount;
+          break;
+        default:
+          throw new AssertionError();
+      }
+      final long maxDoc = writeSparseMissingBitset(values, numberType, numDocsWithValue);
+      assert maxDoc == count;
+    } else if (missingCount == 0) {
+      meta.writeLong(ALL_LIVE);
+    } else if (missingCount == count) {
+      meta.writeLong(ALL_MISSING);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    }
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(count);
+
+    switch (format) {
+      case CONST_COMPRESSED:
+        // write the constant (nonzero value in the n=2 case, singleton value otherwise)
+        meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
+        break;
+      case GCD_COMPRESSED:
+        meta.writeLong(minValue);
+        meta.writeLong(gcd);
+        final long maxDelta = (maxValue - minValue) / gcd;
+        final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
+        meta.writeVInt(bits);
+        final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
+        for (Number nv : values) {
+          long value = nv == null ? 0 : nv.longValue();
+          quotientWriter.add((value - minValue) / gcd);
+        }
+        quotientWriter.finish();
+        break;
+      case DELTA_COMPRESSED:
+        final long minDelta = delta < 0 ? 0 : minValue;
+        meta.writeLong(minDelta);
+        meta.writeVInt(deltaBitsRequired);
+        final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
+        for (Number nv : values) {
+          long v = nv == null ? 0 : nv.longValue();
+          writer.add(v - minDelta);
+        }
+        writer.finish();
+        break;
+      case TABLE_COMPRESSED:
+        final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
+        Arrays.sort(decode);
+        final HashMap<Long,Integer> encode = new HashMap<>();
+        meta.writeVInt(decode.length);
+        for (int i = 0; i < decode.length; i++) {
+          meta.writeLong(decode[i]);
+          encode.put(decode[i], i);
+        }
+        meta.writeVInt(tableBitsRequired);
+        final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
+        for (Number nv : values) {
+          ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
+        }
+        ordsWriter.finish();
+        break;
+      case SPARSE_COMPRESSED:
+        final Iterable<Number> filteredMissingValues;
+        switch (numberType) {
+          case VALUE:
+            meta.writeByte((byte) 0);
+            filteredMissingValues = new Iterable<Number>() {
+              @Override
+              public Iterator<Number> iterator() {
+                return StreamSupport
+                    .stream(values.spliterator(), false)
+                    .filter(value -> value != null)
+                    .iterator();
+              }
+            };
+            break;
+          case ORDINAL:
+            meta.writeByte((byte) 1);
+            filteredMissingValues = new Iterable<Number>() {
+              @Override
+              public Iterator<Number> iterator() {
+                return StreamSupport
+                    .stream(values.spliterator(), false)
+                    .filter(value -> value.longValue() != -1L)
+                    .iterator();
+              }
+            };
+            break;
+          default:
+            throw new AssertionError();
+        }
+        // Write non-missing values as a numeric field
+        addNumericField(field, filteredMissingValues, numberType);
+        break;
+      default:
+        throw new AssertionError();
+    }
+    meta.writeLong(data.getFilePointer());
+  }
+  
+  // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
+  // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
+  void writeMissingBitset(Iterable<?> values) throws IOException {
+    byte bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 8) {
+        data.writeByte(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1 << (count & 7);
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeByte(bits);
+    }
+  }
+
+  long writeSparseMissingBitset(Iterable<Number> values, NumberType numberType, long numDocsWithValue) throws IOException {
+    meta.writeVLong(numDocsWithValue);
+
+    // Write doc IDs that have a value
+    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+    final DirectMonotonicWriter docIdsWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithValue, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long docID = 0;
+    for (Number nv : values) {
+      switch (numberType) {
+        case VALUE:
+          if (nv != null) {
+            docIdsWriter.add(docID);
+          }
+          break;
+        case ORDINAL:
+          if (nv.longValue() != -1L) {
+            docIdsWriter.add(docID);
+          }
+          break;
+        default:
+          throw new AssertionError();
+      }
+      docID++;
+    }
+    docIdsWriter.finish();
+    return docID;
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
+  }
+  
+  private void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.BINARY);
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    final long startFP = data.getFilePointer();
+    long count = 0;
+    long missingCount = 0;
+    for(BytesRef v : values) {
+      final int length;
+      if (v == null) {
+        length = 0;
+        missingCount++;
+      } else {
+        length = v.length;
+      }
+      minLength = Math.min(minLength, length);
+      maxLength = Math.max(maxLength, length);
+      if (v != null) {
+        data.writeBytes(v.bytes, v.offset, v.length);
+      }
+      count++;
+    }
+    meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
+    if (missingCount == 0) {
+      meta.writeLong(ALL_LIVE);
+    } else if (missingCount == count) {
+      meta.writeLong(ALL_MISSING);
+    } else {
+      meta.writeLong(data.getFilePointer());
+      writeMissingBitset(values);
+    }
+    meta.writeVInt(minLength);
+    meta.writeVInt(maxLength);
+    meta.writeVLong(count);
+    meta.writeLong(startFP);
+    
+    // if minLength == maxLength, it's a fixed-length byte[], we are done (the addresses are implicit)
+    // otherwise, we need to record the length fields...
+    if (minLength != maxLength) {
+      meta.writeLong(data.getFilePointer());
+      meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+      final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+      long addr = 0;
+      writer.add(addr);
+      for (BytesRef v : values) {
+        if (v != null) {
+          addr += v.length;
+        }
+        writer.add(addr);
+      }
+      writer.finish();
+      meta.writeLong(data.getFilePointer());
+    }
+  }
+  
+  /** expert: writes a value dictionary for a sorted/sortedset field */
+  private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // first check if it's a "fixed-length" terms dict, and compressibility if so
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    long numValues = 0;
+    BytesRefBuilder previousValue = new BytesRefBuilder();
+    long prefixSum = 0; // only valid for fixed-width data, as we have a choice there
+    for (BytesRef v : values) {
+      minLength = Math.min(minLength, v.length);
+      maxLength = Math.max(maxLength, v.length);
+      if (minLength == maxLength) {
+        int termPosition = (int) (numValues & INTERVAL_MASK);
+        if (termPosition == 0) {
+          // first term in block, save it away to compare against the last term later
+          previousValue.copyBytes(v);
+        } else if (termPosition == INTERVAL_COUNT - 1) {
+          // last term in block, accumulate shared prefix against first term
+          prefixSum += StringHelper.bytesDifference(previousValue.get(), v);
+        }
+      }
+      numValues++;
+    }
+    // for fixed width data, look at the avg(shared prefix) before deciding how to encode:
+    // prefix compression "costs" worst case 2 bytes per term because we must store suffix lengths.
+    // so if we share at least 3 bytes on average, always compress.
+    if (minLength == maxLength && prefixSum <= 3*(numValues >> INTERVAL_SHIFT)) {
+      // no index needed: not very compressible, direct addressing by mult
+      addBinaryField(field, values);
+    } else if (numValues < REVERSE_INTERVAL_COUNT) {
+      // low cardinality: waste a few KB of ram, but can't really use fancy index etc
+      addBinaryField(field, values);
+    } else {
+      assert numValues > 0; // we don't have to handle the empty case
+      // header
+      meta.writeVInt(field.number);
+      meta.writeByte(Lucene54DocValuesFormat.BINARY);
+      meta.writeVInt(BINARY_PREFIX_COMPRESSED);
+      meta.writeLong(-1L);
+      // now write the bytes: sharing prefixes within a block
+      final long startFP = data.getFilePointer();
+      // currently, we have to store the delta from expected for every 1/nth term
+      // we could avoid this, but it's not much and less overall RAM than the previous approach!
+      RAMOutputStream addressBuffer = new RAMOutputStream();
+      MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
+      // buffers up 16 terms
+      RAMOutputStream bytesBuffer = new RAMOutputStream();
+      // buffers up block header
+      RAMOutputStream headerBuffer = new RAMOutputStream();
+      BytesRefBuilder lastTerm = new BytesRefBuilder();
+      lastTerm.grow(maxLength);
+      long count = 0;
+      int suffixDeltas[] = new int[INTERVAL_COUNT];
+      for (BytesRef v : values) {
+        int termPosition = (int) (count & INTERVAL_MASK);
+        if (termPosition == 0) {
+          termAddresses.add(data.getFilePointer() - startFP);
+          // abs-encode first term
+          headerBuffer.writeVInt(v.length);
+          headerBuffer.writeBytes(v.bytes, v.offset, v.length);
+          lastTerm.copyBytes(v);
+        } else {
+          // prefix-code: we only share at most 255 characters, to encode the length as a single
+          // byte and have random access. Larger terms just get less compression.
+          int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
+          bytesBuffer.writeByte((byte) sharedPrefix);
+          bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
+          // we can encode one smaller, because terms are unique.
+          suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
+        }
+        
+        count++;
+        // flush block
+        if ((count & INTERVAL_MASK) == 0) {
+          flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+        }
+      }
+      // flush trailing crap
+      int leftover = (int) (count & INTERVAL_MASK);
+      if (leftover > 0) {
+        Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
+        flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
+      }
+      final long indexStartFP = data.getFilePointer();
+      // write addresses of indexed terms
+      termAddresses.finish();
+      addressBuffer.writeTo(data);
+      addressBuffer = null;
+      termAddresses = null;
+      meta.writeVInt(minLength);
+      meta.writeVInt(maxLength);
+      meta.writeVLong(count);
+      meta.writeLong(startFP);
+      meta.writeLong(indexStartFP);
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(MONOTONIC_BLOCK_SIZE);
+      addReverseTermIndex(field, values, maxLength);
+    }
+  }
+  // writes term dictionary "block"
+  // first term is absolute encoded as vint length + bytes.
+  // lengths of subsequent N terms are encoded as either N bytes or N shorts.
+  // in the double-byte case, the first byte is indicated with -1.
+  // subsequent terms are encoded as byte suffixLength + bytes.
+  private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
+    boolean twoByte = false;
+    for (int i = 1; i < suffixDeltas.length; i++) {
+      if (suffixDeltas[i] > 254) {
+        twoByte = true;
+      }
+    }
+    if (twoByte) {
+      headerBuffer.writeByte((byte)255);
+      for (int i = 1; i < suffixDeltas.length; i++) {
+        headerBuffer.writeShort((short) suffixDeltas[i]);
+      }
+    } else {
+      for (int i = 1; i < suffixDeltas.length; i++) {
+        headerBuffer.writeByte((byte) suffixDeltas[i]);
+      }
+    }
+    headerBuffer.writeTo(data);
+    headerBuffer.reset();
+    bytesBuffer.writeTo(data);
+    bytesBuffer.reset();
+  }
+  
+  // writes reverse term index: used for binary searching a term into a range of 64 blocks
+  // for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
+  // terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
+  private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
+    long count = 0;
+    BytesRefBuilder priorTerm = new BytesRefBuilder();
+    priorTerm.grow(maxLength);
+    BytesRef indexTerm = new BytesRef();
+    long startFP = data.getFilePointer();
+    PagedBytes pagedBytes = new PagedBytes(15);
+    MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
+    
+    for (BytesRef b : values) {
+      int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
+      if (termPosition == 0) {
+        int len = StringHelper.sortKeyLength(priorTerm.get(), b);
+        indexTerm.bytes = b.bytes;
+        indexTerm.offset = b.offset;
+        indexTerm.length = len;
+        addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
+      } else if (termPosition == REVERSE_INTERVAL_MASK) {
+        priorTerm.copyBytes(b);
+      }
+      count++;
+    }
+    addresses.finish();
+    long numBytes = pagedBytes.getPointer();
+    pagedBytes.freeze(true);
+    PagedBytesDataInput in = pagedBytes.getDataInput();
+    meta.writeLong(startFP);
+    data.writeVLong(numBytes);
+    data.copyBytes(in, numBytes);
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED);
+    addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)));
+    addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL);
+  }
+
+  private void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> ords) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED);
+    addTermsDict(field, values);
+    addNumericField(field, ords, NumberType.ORDINAL);
+  }
+
+  @Override
+  public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
+
+    final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
+    final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
+    
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED_NUMERIC);
+    if (isSingleValued(docToValueCount)) {
+      meta.writeVInt(SORTED_SINGLE_VALUED);
+      // The field is single-valued, we can encode it as NUMERIC
+      addNumericField(field, singletonView(docToValueCount, values, null), NumberType.VALUE);
+    } else {
+      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
+      if (uniqueValueSets != null) {
+        meta.writeVInt(SORTED_SET_TABLE);
+
+        // write the set_id -> values mapping
+        writeDictionary(uniqueValueSets);
+
+        // write the doc -> set_id as a numeric field
+        addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), NumberType.ORDINAL);
+      } else {
+        meta.writeVInt(SORTED_WITH_ADDRESSES);
+        // write the stream of values as a numeric field
+        addNumericField(field, values, NumberType.VALUE);
+        // write the doc -> ord count as a absolute index to the stream
+        addOrdIndex(field, docToValueCount);
+      }
+    }
+  }
+
+  @Override
+  public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+
+    Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
+    Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
+    Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
+
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED_SET);
+
+    if (isSingleValued(docToOrdCount)) {
+      meta.writeVInt(SORTED_SINGLE_VALUED);
+      // The field is single-valued, we can encode it as SORTED
+      addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
+    } else {
+      final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
+      if (uniqueValueSets != null) {
+        meta.writeVInt(SORTED_SET_TABLE);
+
+        // write the set_id -> ords mapping
+        writeDictionary(uniqueValueSets);
+
+        // write the ord -> byte[] as a binary field
+        addTermsDict(field, values);
+
+        // write the doc -> set_id as a numeric field
+        addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), NumberType.ORDINAL);
+      } else {
+        meta.writeVInt(SORTED_WITH_ADDRESSES);
+
+        // write the ord -> byte[] as a binary field
+        addTermsDict(field, values);
+
+        // write the stream of ords as a numeric field
+        // NOTE: we could return an iterator that delta-encodes these within a doc
+        addNumericField(field, ords, NumberType.ORDINAL);
+
+        // write the doc -> ord count as a absolute index to the stream
+        addOrdIndex(field, docToOrdCount);
+      }
+    }
+  }
+
+  private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
+    Set<LongsRef> uniqueValueSet = new HashSet<>();
+    LongsRef docValues = new LongsRef(256);
+
+    Iterator<Number> valueCountIterator = docToValueCount.iterator();
+    Iterator<Number> valueIterator = values.iterator();
+    int totalDictSize = 0;
+    while (valueCountIterator.hasNext()) {
+      docValues.length = valueCountIterator.next().intValue();
+      if (docValues.length > 256) {
+        return null;
+      }
+      for (int i = 0; i < docValues.length; ++i) {
+        docValues.longs[i] = valueIterator.next().longValue();
+      }
+      if (uniqueValueSet.contains(docValues)) {
+        continue;
+      }
+      totalDictSize += docValues.length;
+      if (totalDictSize > 256) {
+        return null;
+      }
+      uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
+    }
+    assert valueIterator.hasNext() == false;
+    return new TreeSet<>(uniqueValueSet);
+  }
+
+  private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
+    int lengthSum = 0;
+    for (LongsRef longs : uniqueValueSets) {
+      lengthSum += longs.length;
+    }
+
+    meta.writeInt(lengthSum);
+    for (LongsRef valueSet : uniqueValueSets) {
+      for (int  i = 0; i < valueSet.length; ++i) {
+        meta.writeLong(valueSet.longs[valueSet.offset + i]);
+      }
+    }
+
+    meta.writeInt(uniqueValueSets.size());
+    for (LongsRef valueSet : uniqueValueSets) {
+      meta.writeInt(valueSet.length);
+    }
+  }
+
+  private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
+    final Map<LongsRef, Integer> setIds = new HashMap<>();
+    int i = 0;
+    for (LongsRef set : uniqueValueSets) {
+      setIds.put(set, i++);
+    }
+    assert i == uniqueValueSets.size();
+
+    return new Iterable<Number>() {
+
+      @Override
+      public Iterator<Number> iterator() {
+        final Iterator<Number> valueCountIterator = docToValueCount.iterator();
+        final Iterator<Number> valueIterator = values.iterator();
+        final LongsRef docValues = new LongsRef(256);
+        return new Iterator<Number>() {
+
+          @Override
+          public boolean hasNext() {
+            return valueCountIterator.hasNext();
+          }
+
+          @Override
+          public Number next() {
+            docValues.length = valueCountIterator.next().intValue();
+            for (int i = 0; i < docValues.length; ++i) {
+              docValues.longs[i] = valueIterator.next().longValue();
+            }
+            final Integer id = setIds.get(docValues);
+            assert id != null;
+            return id;
+          }
+
+        };
+
+      }
+    };
+  }
+
+  // writes addressing information as MONOTONIC_COMPRESSED integer
+  private void addOrdIndex(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.NUMERIC);
+    meta.writeVInt(MONOTONIC_COMPRESSED);
+    meta.writeLong(-1L);
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(maxDoc);
+    meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
+
+    final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    long addr = 0;
+    writer.add(addr);
+    for (Number v : values) {
+      addr += v.longValue();
+      writer.add(addr);
+    }
+    writer.finish();
+    meta.writeLong(data.getFilePointer());
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
new file mode 100644
index 0000000..91ccfe2
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene54;
+
+
+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.codecs.lucene70.Lucene70DocValuesFormat;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.SmallFloat;
+import org.apache.lucene.util.packed.DirectWriter;
+
+/**
+ * Lucene 5.4 DocValues format.
+ * <p>
+ * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these 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 non-missing value, only the missing
+ *        bitset is encoded.
+ *    <li>Sparse-compressed: only documents with a value are stored, and lookups are performed
+ *        using binary search.
+ * </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 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 table: when there are few unique sets of values (&lt; 256) then each set is assigned
+ *        an id, a lookup table is written and the mapping from document to set id is written using the
+ *        numeric strategies above.
+ *    <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>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
+ *        an id, a lookup table is written and the mapping from document to set id is written using the
+ *        numeric strategies above.
+ *    <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
+ *        strategies above.
+ * </ul>
+ * <p>
+ * Files:
+ * <ol>
+ *   <li><tt>.dvd</tt>: DocValues data</li>
+ *   <li><tt>.dvm</tt>: DocValues metadata</li>
+ * </ol>
+ * @lucene.experimental
+ * @deprecated Use {@link Lucene70DocValuesFormat}.
+ */
+@Deprecated
+public final class Lucene54DocValuesFormat extends DocValuesFormat {
+
+  /** Sole Constructor */
+  public Lucene54DocValuesFormat() {
+    super("Lucene54");
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene54DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new Lucene54DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "Lucene54DocValuesData";
+  static final String DATA_EXTENSION = "dvd";
+  static final String META_CODEC = "Lucene54DocValuesMetadata";
+  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;
+  
+  // address terms in blocks of 16 terms
+  static final int INTERVAL_SHIFT = 4;
+  static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
+  static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
+  
+  // build reverse index from every 1024th term
+  static final int REVERSE_INTERVAL_SHIFT = 10;
+  static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
+  static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
+  
+  // for conversion from reverse index to block
+  static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
+  static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
+  static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
+
+  /** Compressed using packed blocks of ints. */
+  static final int DELTA_COMPRESSED = 0;
+  /** Compressed by computing the GCD. */
+  static final int GCD_COMPRESSED = 1;
+  /** Compressed by giving IDs to unique values. */
+  static final int TABLE_COMPRESSED = 2;
+  /** Compressed with monotonically increasing values */
+  static final int MONOTONIC_COMPRESSED = 3;
+  /** Compressed with constant value (uses only missing bitset) */
+  static final int CONST_COMPRESSED = 4;
+  /** Compressed with sparse arrays. */
+  static final int SPARSE_COMPRESSED = 5;
+
+  /** Uncompressed binary, written directly (fixed length). */
+  static final int BINARY_FIXED_UNCOMPRESSED = 0;
+  /** Uncompressed binary, written directly (variable length). */
+  static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
+  /** Compressed binary with shared prefixes */
+  static final int BINARY_PREFIX_COMPRESSED = 2;
+
+  /** Standard storage for sorted set values with 1 level of indirection:
+   *  {@code docId -> address -> ord}. */
+  static final int SORTED_WITH_ADDRESSES = 0;
+  /** Single-valued sorted set values, encoded as sorted values, so no level
+   *  of indirection: {@code docId -> ord}. */
+  static final int SORTED_SINGLE_VALUED = 1;
+  /** Compressed giving IDs to unique sets of values:
+   * {@code docId -> setId -> ords} */
+  static final int SORTED_SET_TABLE = 2;
+  
+  /** placeholder for missing offset that means there are no missing values */
+  static final int ALL_LIVE = -1;
+  /** placeholder for missing offset that means all values are missing */
+  static final int ALL_MISSING = -2;
+  
+  // addressing uses 16k blocks
+  static final int MONOTONIC_BLOCK_SIZE = 16384;
+  static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
+}


[2/6] lucene-solr:master: LUCENE-7475: Make norms sparse.

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
deleted file mode 100644
index 9356aed..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
+++ /dev/null
@@ -1,1707 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene54;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.lucene54.Lucene54DocValuesConsumer.NumberType;
-import org.apache.lucene.index.*;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RandomAccessInput;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LongValues;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.DirectMonotonicReader;
-import org.apache.lucene.util.packed.DirectReader;
-import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
-
-import static org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat.*;
-
-/** reader for {@link Lucene54DocValuesFormat} */
-final class Lucene54DocValuesProducer extends DocValuesProducer implements Closeable {
-  private final Map<String,NumericEntry> numerics = new HashMap<>();
-  private final Map<String,BinaryEntry> binaries = new HashMap<>();
-  private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
-  private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
-  private final Map<String,NumericEntry> ords = new HashMap<>();
-  private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
-  private final int numFields;
-  private final AtomicLong ramBytesUsed;
-  private final IndexInput data;
-  private final int maxDoc;
-
-  // memory-resident structures
-  private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
-  private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
-  private final Map<String,DirectMonotonicReader.Meta> directAddressesMeta = new HashMap<>();
-
-  private final boolean merging;
-
-  // clone for merge: when merging we don't do any instances.put()s
-  Lucene54DocValuesProducer(Lucene54DocValuesProducer original) throws IOException {
-    assert Thread.holdsLock(original);
-    numerics.putAll(original.numerics);
-    binaries.putAll(original.binaries);
-    sortedSets.putAll(original.sortedSets);
-    sortedNumerics.putAll(original.sortedNumerics);
-    ords.putAll(original.ords);
-    ordIndexes.putAll(original.ordIndexes);
-    numFields = original.numFields;
-    ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
-    data = original.data.clone();
-    maxDoc = original.maxDoc;
-
-    addressInstances.putAll(original.addressInstances);
-    reverseIndexInstances.putAll(original.reverseIndexInstances);
-    merging = true;
-  }
-
-  /** expert: instantiates a new reader */
-  Lucene54DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
-    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
-    this.maxDoc = state.segmentInfo.maxDoc();
-    merging = false;
-    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
-
-    int version = -1;
-    int numFields = -1;
-
-    // read in the entries from the metadata file.
-    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
-      Throwable priorE = null;
-      try {
-        version = CodecUtil.checkIndexHeader(in, metaCodec,
-                                        Lucene54DocValuesFormat.VERSION_START,
-                                        Lucene54DocValuesFormat.VERSION_CURRENT,
-                                        state.segmentInfo.getId(),
-                                        state.segmentSuffix);
-        numFields = readFields(in, state.fieldInfos);
-      } catch (Throwable exception) {
-        priorE = exception;
-      } finally {
-        CodecUtil.checkFooter(in, priorE);
-      }
-    }
-
-    this.numFields = numFields;
-    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
-    this.data = state.directory.openInput(dataName, state.context);
-    boolean success = false;
-    try {
-      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec,
-                                                 Lucene54DocValuesFormat.VERSION_START,
-                                                 Lucene54DocValuesFormat.VERSION_CURRENT,
-                                                 state.segmentInfo.getId(),
-                                                 state.segmentSuffix);
-      if (version != version2) {
-        throw new CorruptIndexException("Format versions mismatch: meta=" + version + ", data=" + version2, data);
-      }
-
-      // NOTE: data file is too costly to verify checksum against all the bytes on open,
-      // but for now we at least verify proper structure of the checksum footer: which looks
-      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
-      // such as file truncation.
-      CodecUtil.retrieveChecksum(data);
-
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(this.data);
-      }
-    }
-  }
-
-  private void readSortedField(FieldInfo info, IndexInput meta) throws IOException {
-    // sorted = binary + numeric
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sorted entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n = readNumericEntry(info, meta);
-    ords.put(info.name, n);
-  }
-
-  private void readSortedSetFieldWithAddresses(FieldInfo info, IndexInput meta) throws IOException {
-    // sortedset = binary + numeric (addresses) + ordIndex
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n1 = readNumericEntry(info, meta);
-    ords.put(info.name, n1);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n2 = readNumericEntry(info, meta);
-    ordIndexes.put(info.name, n2);
-  }
-
-  private void readSortedSetFieldWithTable(FieldInfo info, IndexInput meta) throws IOException {
-    // sortedset table = binary + ordset table + ordset index
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.BINARY) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-
-    BinaryEntry b = readBinaryEntry(info, meta);
-    binaries.put(info.name, b);
-
-    if (meta.readVInt() != info.number) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-      throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-    }
-    NumericEntry n = readNumericEntry(info, meta);
-    ords.put(info.name, n);
-  }
-
-  private int readFields(IndexInput meta, FieldInfos infos) throws IOException {
-    int numFields = 0;
-    int fieldNumber = meta.readVInt();
-    while (fieldNumber != -1) {
-      numFields++;
-      FieldInfo info = infos.fieldInfo(fieldNumber);
-      if (info == null) {
-        // trickier to validate more: because we use multiple entries for "composite" types like sortedset, etc.
-        throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
-      }
-      byte type = meta.readByte();
-      if (type == Lucene54DocValuesFormat.NUMERIC) {
-        numerics.put(info.name, readNumericEntry(info, meta));
-      } else if (type == Lucene54DocValuesFormat.BINARY) {
-        BinaryEntry b = readBinaryEntry(info, meta);
-        binaries.put(info.name, b);
-      } else if (type == Lucene54DocValuesFormat.SORTED) {
-        readSortedField(info, meta);
-      } else if (type == Lucene54DocValuesFormat.SORTED_SET) {
-        SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedSets.put(info.name, ss);
-        if (ss.format == SORTED_WITH_ADDRESSES) {
-          readSortedSetFieldWithAddresses(info, meta);
-        } else if (ss.format == SORTED_SET_TABLE) {
-          readSortedSetFieldWithTable(info, meta);
-        } else if (ss.format == SORTED_SINGLE_VALUED) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.SORTED) {
-            throw new CorruptIndexException("sortedset entry for field: " + info.name + " is corrupt", meta);
-          }
-          readSortedField(info, meta);
-        } else {
-          throw new AssertionError();
-        }
-      } else if (type == Lucene54DocValuesFormat.SORTED_NUMERIC) {
-        SortedSetEntry ss = readSortedSetEntry(meta);
-        sortedNumerics.put(info.name, ss);
-        if (ss.format == SORTED_WITH_ADDRESSES) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          numerics.put(info.name, readNumericEntry(info, meta));
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          NumericEntry ordIndex = readNumericEntry(info, meta);
-          ordIndexes.put(info.name, ordIndex);
-        } else if (ss.format == SORTED_SET_TABLE) {
-          if (meta.readVInt() != info.number) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          NumericEntry n = readNumericEntry(info, meta);
-          ords.put(info.name, n);
-        } else if (ss.format == SORTED_SINGLE_VALUED) {
-          if (meta.readVInt() != fieldNumber) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          if (meta.readByte() != Lucene54DocValuesFormat.NUMERIC) {
-            throw new CorruptIndexException("sortednumeric entry for field: " + info.name + " is corrupt", meta);
-          }
-          numerics.put(info.name, readNumericEntry(info, meta));
-        } else {
-          throw new AssertionError();
-        }
-      } else {
-        throw new CorruptIndexException("invalid type: " + type, meta);
-      }
-      fieldNumber = meta.readVInt();
-    }
-    return numFields;
-  }
-
-  private NumericEntry readNumericEntry(FieldInfo info, IndexInput meta) throws IOException {
-    NumericEntry entry = new NumericEntry();
-    entry.format = meta.readVInt();
-    entry.missingOffset = meta.readLong();
-    if (entry.format == SPARSE_COMPRESSED) {
-      // sparse bits need a bit more metadata
-      entry.numDocsWithValue = meta.readVLong();
-      final int blockShift = meta.readVInt();
-      entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, entry.numDocsWithValue, blockShift);
-      ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
-      directAddressesMeta.put(info.name, entry.monotonicMeta);
-    }
-    entry.offset = meta.readLong();
-    entry.count = meta.readVLong();
-    switch(entry.format) {
-      case CONST_COMPRESSED:
-        entry.minValue = meta.readLong();
-        if (entry.count > Integer.MAX_VALUE) {
-          // currently just a limitation e.g. of bits interface and so on.
-          throw new CorruptIndexException("illegal CONST_COMPRESSED count: " + entry.count, meta);
-        }
-        break;
-      case GCD_COMPRESSED:
-        entry.minValue = meta.readLong();
-        entry.gcd = meta.readLong();
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case TABLE_COMPRESSED:
-        final int uniqueValues = meta.readVInt();
-        if (uniqueValues > 256) {
-          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + uniqueValues, meta);
-        }
-        entry.table = new long[uniqueValues];
-        for (int i = 0; i < uniqueValues; ++i) {
-          entry.table[i] = meta.readLong();
-        }
-        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case DELTA_COMPRESSED:
-        entry.minValue = meta.readLong();
-        entry.bitsPerValue = meta.readVInt();
-        break;
-      case MONOTONIC_COMPRESSED:
-        final int blockShift = meta.readVInt();
-        entry.monotonicMeta = DirectMonotonicReader.loadMeta(meta, maxDoc + 1, blockShift);
-        ramBytesUsed.addAndGet(entry.monotonicMeta.ramBytesUsed());
-        directAddressesMeta.put(info.name, entry.monotonicMeta);
-        break;
-      case SPARSE_COMPRESSED:
-        final byte numberType = meta.readByte();
-        switch (numberType) {
-          case 0:
-            entry.numberType = NumberType.VALUE;
-            break;
-          case 1:
-            entry.numberType = NumberType.ORDINAL;
-            break;
-          default:
-            throw new CorruptIndexException("Number type can only be 0 or 1, got=" + numberType, meta);
-        }
-
-        // now read the numeric entry for non-missing values
-        final int fieldNumber = meta.readVInt();
-        if (fieldNumber != info.number) {
-          throw new CorruptIndexException("Field numbers mistmatch: " + fieldNumber + " != " + info.number, meta);
-        }
-        final int dvFormat = meta.readByte();
-        if (dvFormat != NUMERIC) {
-          throw new CorruptIndexException("Formats mistmatch: " + dvFormat + " != " + NUMERIC, meta);
-        }
-        entry.nonMissingValues = readNumericEntry(info, meta);
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=", meta);
-    }
-    entry.endOffset = meta.readLong();
-    return entry;
-  }
-
-  private BinaryEntry readBinaryEntry(FieldInfo info, IndexInput meta) throws IOException {
-    BinaryEntry entry = new BinaryEntry();
-    entry.format = meta.readVInt();
-    entry.missingOffset = meta.readLong();
-    entry.minLength = meta.readVInt();
-    entry.maxLength = meta.readVInt();
-    entry.count = meta.readVLong();
-    entry.offset = meta.readLong();
-    switch(entry.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        break;
-      case BINARY_PREFIX_COMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        entry.packedIntsVersion = meta.readVInt();
-        entry.blockSize = meta.readVInt();
-        entry.reverseIndexOffset = meta.readLong();
-        break;
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        final int blockShift = meta.readVInt();
-        entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, entry.count + 1, blockShift);
-        ramBytesUsed.addAndGet(entry.addressesMeta.ramBytesUsed());
-        directAddressesMeta.put(info.name, entry.addressesMeta);
-        entry.addressesEndOffset = meta.readLong();
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format, meta);
-    }
-    return entry;
-  }
-
-  SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
-    SortedSetEntry entry = new SortedSetEntry();
-    entry.format = meta.readVInt();
-    if (entry.format == SORTED_SET_TABLE) {
-      final int totalTableLength = meta.readInt();
-      if (totalTableLength > 256) {
-        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more than 256 values in its dictionary, got=" + totalTableLength, meta);
-      }
-      entry.table = new long[totalTableLength];
-      for (int i = 0; i < totalTableLength; ++i) {
-        entry.table[i] = meta.readLong();
-      }
-      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.table));
-      final int tableSize = meta.readInt();
-      if (tableSize > totalTableLength + 1) { // +1 because of the empty set
-        throw new CorruptIndexException("SORTED_SET_TABLE cannot have more set ids than ords in its dictionary, got " + totalTableLength + " ords and " + tableSize + " sets", meta);
-      }
-      entry.tableOffsets = new int[tableSize + 1];
-      for (int i = 1; i < entry.tableOffsets.length; ++i) {
-        entry.tableOffsets[i] = entry.tableOffsets[i - 1] + meta.readInt();
-      }
-      ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(entry.tableOffsets));
-    } else if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
-      throw new CorruptIndexException("Unknown format: " + entry.format, meta);
-    }
-    return entry;
-  }
-
-  @Override
-  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.name);
-    Bits docsWithField;
-
-    if (entry.format == SPARSE_COMPRESSED) {
-      return getSparseNumericDocValues(entry);
-    } else {
-      if (entry.missingOffset == ALL_MISSING) {
-        return DocValues.emptyNumeric();
-      } else if (entry.missingOffset == ALL_LIVE) {
-        LongValues values = getNumeric(entry);
-        return new NumericDocValues() {
-          private int docID = -1;
-
-          @Override
-          public int docID() {
-            return docID;
-          }
-
-          @Override
-          public int nextDoc() {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-            }
-            return docID;
-          }
-
-          @Override
-          public int advance(int target) {
-            if (target >= maxDoc) {
-              docID = NO_MORE_DOCS;
-            } else {
-              docID = target;
-            }
-            return docID;
-          }
-
-          @Override
-          public long cost() {
-            // TODO
-            return 0;
-          }
-
-          @Override
-          public long longValue() {
-            return values.get(docID);
-          }
-        };
-      } else {
-        docsWithField = getLiveBits(entry.missingOffset, maxDoc);
-      }
-    }
-    return new LegacyNumericDocValuesWrapper(docsWithField, getNumeric(entry));
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return ramBytesUsed.get();
-  }
-
-  @Override
-  public synchronized Collection<Accountable> getChildResources() {
-    List<Accountable> resources = new ArrayList<>();
-    resources.addAll(Accountables.namedAccountables("addresses field", addressInstances));
-    resources.addAll(Accountables.namedAccountables("reverse index field", reverseIndexInstances));
-    resources.addAll(Accountables.namedAccountables("direct addresses meta field", directAddressesMeta));
-    return Collections.unmodifiableList(resources);
-  }
-
-  @Override
-  public void checkIntegrity() throws IOException {
-    CodecUtil.checksumEntireFile(data);
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(fields=" + numFields + ")";
-  }
-
-  LongValues getNumeric(NumericEntry entry) throws IOException {
-    switch (entry.format) {
-      case CONST_COMPRESSED: {
-        final long constant = entry.minValue;
-        final Bits live = getLiveBits(entry.missingOffset, (int)entry.count);
-        return new LongValues() {
-          @Override
-          public long get(long index) {
-            return live.get((int)index) ? constant : 0;
-          }
-        };
-      }
-      case DELTA_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long delta = entry.minValue;
-        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return delta + values.get(id);
-          }
-        };
-      }
-      case GCD_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long min = entry.minValue;
-        final long mult = entry.gcd;
-        final LongValues quotientReader = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return min + mult * quotientReader.get(id);
-          }
-        };
-      }
-      case TABLE_COMPRESSED: {
-        RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-        final long table[] = entry.table;
-        final LongValues ords = DirectReader.getInstance(slice, entry.bitsPerValue, 0);
-        return new LongValues() {
-          @Override
-          public long get(long id) {
-            return table[(int) ords.get(id)];
-          }
-        };
-      }
-      case SPARSE_COMPRESSED:
-        final SparseNumericDocValues values = getSparseNumericDocValues(entry);
-        final long missingValue;
-        switch (entry.numberType) {
-          case ORDINAL:
-            missingValue = -1L;
-            break;
-          case VALUE:
-            missingValue = 0L;
-            break;
-          default:
-            throw new AssertionError();
-        }
-        return new SparseNumericDocValuesRandomAccessWrapper(values, missingValue);
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  static final class SparseNumericDocValues extends NumericDocValues {
-
-    final int docIDsLength;
-    final LongValues docIds, values;
-
-    int index, doc;
-
-    SparseNumericDocValues(int docIDsLength, LongValues docIDs, LongValues values) {
-      this.docIDsLength = docIDsLength;
-      this.docIds = docIDs;
-      this.values = values;
-      reset();
-    }
-
-    void reset() {
-      index = -1;
-      doc = -1;
-    }
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      if (index >= docIDsLength - 1) {
-        index = docIDsLength;
-        return doc = NO_MORE_DOCS;
-      }
-      return doc = (int) docIds.get(++index);
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      long loIndex = index;
-      long step = 1;
-      long hiIndex;
-      int hiDoc;
-
-      // gallop forward by exponentially growing the interval
-      // in order to find an interval so that the target doc
-      // is in ]lo, hi]. Compared to a regular binary search,
-      // this optimizes the case that the caller performs many
-      // advance calls by small deltas
-      do {
-        hiIndex = index + step;
-        if (hiIndex >= docIDsLength) {
-          hiIndex = docIDsLength;
-          hiDoc = NO_MORE_DOCS;
-          break;
-        }
-        hiDoc = (int) docIds.get(hiIndex);
-        if (hiDoc >= target) {
-          break;
-        }
-        step <<= 1;
-      } while (true);
-
-      // now binary search
-      while (loIndex + 1 < hiIndex) {
-        final long midIndex = (loIndex + 1 + hiIndex) >>> 1;
-        final int midDoc = (int) docIds.get(midIndex);
-        if (midDoc >= target) {
-          hiIndex = midIndex;
-          hiDoc = midDoc;
-        } else {
-          loIndex = midIndex;
-        }
-      }
-
-      index = (int) hiIndex;
-      return doc = hiDoc;
-    }
-
-    @Override
-    public long longValue() {
-      assert index >= 0;
-      assert index < docIDsLength;
-      return values.get(index);
-    }
-
-    @Override
-    public long cost() {
-      return docIDsLength;
-    }
-  }
-
-  static class SparseNumericDocValuesRandomAccessWrapper extends LongValues {
-
-    final SparseNumericDocValues values;
-    final long missingValue;
-
-    SparseNumericDocValuesRandomAccessWrapper(SparseNumericDocValues values, long missingValue) {
-      this.values = values;
-      this.missingValue = missingValue;
-    }
-
-    @Override
-    public long get(long longIndex) {
-      final int index = Math.toIntExact(longIndex);
-      int doc = values.docID();
-      if (doc >= index) {
-        values.reset();
-      }
-      assert values.docID() < index;
-      try {
-        doc = values.advance(index);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      if (doc == index) {
-        return values.longValue();
-      } else {
-        return missingValue;
-      }
-    }
-
-  }
-
-  LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException {
-    BinaryEntry bytes = binaries.get(field.name);
-    switch(bytes.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        return getFixedBinary(field, bytes);
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        return getVariableBinary(field, bytes);
-      case BINARY_PREFIX_COMPRESSED:
-        return getCompressedBinary(field, bytes);
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  @Override
-  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryEntry be = binaries.get(field.name);
-    return new LegacyBinaryDocValuesWrapper(getLiveBits(be.missingOffset, maxDoc), getLegacyBinary(field));
-  }
-
-  private LegacyBinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
-
-    final BytesRef term = new BytesRef(bytes.maxLength);
-    final byte[] buffer = term.bytes;
-    final int length = term.length = bytes.maxLength;
-
-    return new LongBinaryDocValues() {
-      @Override
-      public BytesRef get(long id) {
-        try {
-          data.seek(id * length);
-          data.readBytes(buffer, 0, buffer.length);
-          return term;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  private LegacyBinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final RandomAccessInput addressesData = this.data.randomAccessSlice(bytes.addressesOffset, bytes.addressesEndOffset - bytes.addressesOffset);
-    final LongValues addresses = DirectMonotonicReader.getInstance(bytes.addressesMeta, addressesData);
-
-    final IndexInput data = this.data.slice("var-binary", bytes.offset, bytes.addressesOffset - bytes.offset);
-    final BytesRef term = new BytesRef(Math.max(0, bytes.maxLength));
-    final byte buffer[] = term.bytes;
-
-    return new LongBinaryDocValues() {
-      @Override
-      public BytesRef get(long id) {
-        long startAddress = addresses.get(id);
-        long endAddress = addresses.get(id+1);
-        int length = (int) (endAddress - startAddress);
-        try {
-          data.seek(startAddress);
-          data.readBytes(buffer, 0, length);
-          term.length = length;
-          return term;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  /** returns an address instance for prefix-compressed binary values. */
-  private synchronized MonotonicBlockPackedReader getIntervalInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    MonotonicBlockPackedReader addresses = addressInstances.get(field.name);
-    if (addresses == null) {
-      data.seek(bytes.addressesOffset);
-      final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
-      addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      if (!merging) {
-        addressInstances.put(field.name, addresses);
-        ramBytesUsed.addAndGet(addresses.ramBytesUsed() + Integer.BYTES);
-      }
-    }
-    return addresses;
-  }
-
-  /** returns a reverse lookup instance for prefix-compressed binary values. */
-  private synchronized ReverseTermsIndex getReverseIndexInstance(FieldInfo field, BinaryEntry bytes) throws IOException {
-    ReverseTermsIndex index = reverseIndexInstances.get(field.name);
-    if (index == null) {
-      index = new ReverseTermsIndex();
-      data.seek(bytes.reverseIndexOffset);
-      long size = (bytes.count + REVERSE_INTERVAL_MASK) >>> REVERSE_INTERVAL_SHIFT;
-      index.termAddresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      long dataSize = data.readVLong();
-      PagedBytes pagedBytes = new PagedBytes(15);
-      pagedBytes.copy(data, dataSize);
-      index.terms = pagedBytes.freeze(true);
-      if (!merging) {
-        reverseIndexInstances.put(field.name, index);
-        ramBytesUsed.addAndGet(index.ramBytesUsed());
-      }
-    }
-    return index;
-  }
-
-  private LegacyBinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
-    final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
-    assert addresses.size() > 0; // we don't have to handle empty case
-    IndexInput slice = data.slice("terms", bytes.offset, bytes.addressesOffset - bytes.offset);
-    return new CompressedBinaryDocValues(bytes, addresses, index, slice);
-  }
-
-  @Override
-  public SortedDocValues getSorted(FieldInfo field) throws IOException {
-    final int valueCount = (int) binaries.get(field.name).count;
-    final LegacyBinaryDocValues binary = getLegacyBinary(field);
-    NumericEntry entry = ords.get(field.name);
-    final LongValues ordinals = getNumeric(entry);
-    if (entry.format == SPARSE_COMPRESSED) {
-      final SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) ordinals).values;
-      return new SortedDocValues() {
-
-        @Override
-        public int ordValue() {
-          return (int) sparseValues.longValue();
-        }
-
-        @Override
-        public BytesRef lookupOrd(int ord) {
-          return binary.get(ord);
-        }
-
-        @Override
-        public int getValueCount() {
-          return valueCount;
-        }
-
-        @Override
-        public int docID() {
-          return sparseValues.docID();
-        }
-
-        @Override
-        public int nextDoc() throws IOException {
-          return sparseValues.nextDoc();
-        }
-
-        @Override
-        public int advance(int target) throws IOException {
-          return sparseValues.advance(target);
-        }
-
-        @Override
-        public long cost() {
-          return sparseValues.cost();
-        }
-
-      };
-    }
-    return new SortedDocValues() {
-      private int docID = -1;
-      private int ord;
-
-      @Override
-      public int docID() {
-        return docID;
-      }
-
-      @Override
-      public int nextDoc() throws IOException {
-        assert docID != NO_MORE_DOCS;
-        while (true) {
-          docID++;
-          if (docID == maxDoc) {
-            docID = NO_MORE_DOCS;
-            break;
-          }
-          ord = (int) ordinals.get(docID);
-          if (ord != -1) {
-            break;
-          }
-        }
-        return docID;
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        if (target >= maxDoc) {
-          docID = NO_MORE_DOCS;
-          return docID;
-        } else {
-          docID = target-1;
-          return nextDoc();
-        }
-      }
-          
-      @Override
-      public int ordValue() {
-        return ord;
-      }
-
-      @Override
-      public long cost() {
-        // TODO
-        return 0;
-      }
-
-      @Override
-      public BytesRef lookupOrd(int ord) {
-        return binary.get(ord);
-      }
-
-      @Override
-      public int getValueCount() {
-        return valueCount;
-      }
-
-      @Override
-      public int lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-    };
-  }
-
-  /** returns an address instance for sortedset ordinal lists */
-  private LongValues getOrdIndexInstance(FieldInfo field, NumericEntry entry) throws IOException {
-    RandomAccessInput data = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-    return DirectMonotonicReader.getInstance(entry.monotonicMeta, data);
-  }
-
-  @Override
-  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedNumerics.get(field.name);
-    if (ss.format == SORTED_SINGLE_VALUED) {
-      NumericEntry numericEntry = numerics.get(field.name);
-      final LongValues values = getNumeric(numericEntry);
-      if (numericEntry.format == SPARSE_COMPRESSED) {
-        SparseNumericDocValues sparseValues = ((SparseNumericDocValuesRandomAccessWrapper) values).values;
-        return new SortedNumericDocValues() {
-
-          @Override
-          public long nextValue() throws IOException {
-            return sparseValues.longValue();
-          }
-
-          @Override
-          public int docValueCount() {
-            return 1;
-          }
-
-          @Override
-          public int docID() {
-            return sparseValues.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return sparseValues.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return sparseValues.advance(target);
-          }
-
-          @Override
-          public long cost() {
-            return sparseValues.cost();
-          }
-
-        };
-      }
-      final Bits docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
-      return new SortedNumericDocValues() {
-        int docID = -1;
-
-        @Override
-        public int docID() {
-          return docID;
-        }
-
-        @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              break;
-            }
-
-            if (docsWithField.get(docID)) {
-              // TODO: use .nextSetBit here, at least!!
-              break;
-            }
-          }
-          return docID;
-        }
-
-        @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-
-        @Override
-        public long cost() {
-          // TODO
-          return 0;
-        }
-
-        @Override
-        public int docValueCount() {
-          return 1;
-        }
-
-        @Override
-        public long nextValue() {
-          return values.get(docID);
-        }
-      };
-    } else if (ss.format == SORTED_WITH_ADDRESSES) {
-      NumericEntry numericEntry = numerics.get(field.name);
-      final LongValues values = getNumeric(numericEntry);
-      final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
-
-      return new SortedNumericDocValues() {
-        long startOffset;
-        long endOffset;
-        int docID = -1;
-        long upto;
-
-        @Override
-        public int docID() {
-          return docID;
-        }
-
-        @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              return docID;
-            }
-            startOffset = ordIndex.get(docID);
-            endOffset = ordIndex.get(docID+1L);
-            if (endOffset > startOffset) {
-              break;
-            }
-          }
-          upto = startOffset;
-          return docID;
-        }
-
-        @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-        
-        @Override
-        public long cost() {
-          // TODO
-          return 0;
-        }
-        
-        @Override
-        public int docValueCount() {
-          return (int) (endOffset - startOffset);
-        }
-        
-        @Override
-        public long nextValue() {
-          return values.get(upto++);
-        }
-      };
-    } else if (ss.format == SORTED_SET_TABLE) {
-      NumericEntry entry = ords.get(field.name);
-      final LongValues ordinals = getNumeric(entry);
-
-      final long[] table = ss.table;
-      final int[] offsets = ss.tableOffsets;
-      return new SortedNumericDocValues() {
-        int startOffset;
-        int endOffset;
-        int docID = -1;
-        int upto;
-
-        @Override
-        public int docID() {
-          return docID;
-        }
-
-        @Override
-        public int nextDoc() {
-          while (true) {
-            docID++;
-            if (docID == maxDoc) {
-              docID = NO_MORE_DOCS;
-              return docID;
-            }
-            int ord = (int) ordinals.get(docID);
-            startOffset = offsets[ord];
-            endOffset = offsets[ord+1];
-            if (endOffset > startOffset) {
-              break;
-            }
-          }
-          upto = startOffset;
-          return docID;
-        }
-
-        @Override
-        public int advance(int target) {
-          if (target >= maxDoc) {
-            docID = NO_MORE_DOCS;
-            return docID;
-          } else {
-            docID = target-1;
-            return nextDoc();
-          }
-        }
-        
-        @Override
-        public long cost() {
-          // TODO
-          return 0;
-        }
-
-        @Override
-        public int docValueCount() {
-          return endOffset - startOffset;
-        }
-        
-        @Override
-        public long nextValue() {
-          return table[upto++];
-        }
-      };
-    } else {
-      throw new AssertionError();
-    }
-  }
-
-  @Override
-  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
-    SortedSetEntry ss = sortedSets.get(field.name);
-    switch (ss.format) {
-      case SORTED_SINGLE_VALUED:
-        return DocValues.singleton(getSorted(field));
-      case SORTED_WITH_ADDRESSES:
-        return getSortedSetWithAddresses(field);
-      case SORTED_SET_TABLE:
-        return getSortedSetTable(field, ss);
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
-    final long valueCount = binaries.get(field.name).count;
-    // we keep the byte[]s and list of ords on disk, these could be large
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
-    final LongValues ordinals = getNumeric(ords.get(field.name));
-    // but the addresses to the ord stream are in RAM
-    final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
-
-    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
-      long startOffset;
-      long offset;
-      long endOffset;
-
-      @Override
-      public long nextOrd() {
-        if (offset == endOffset) {
-          return NO_MORE_ORDS;
-        } else {
-          long ord = ordinals.get(offset);
-          offset++;
-          return ord;
-        }
-      }
-
-      @Override
-      public void setDocument(int docID) {
-        startOffset = offset = ordIndex.get(docID);
-        endOffset = ordIndex.get(docID+1L);
-      }
-
-      @Override
-      public BytesRef lookupOrd(long ord) {
-        return binary.get(ord);
-      }
-
-      @Override
-      public long getValueCount() {
-        return valueCount;
-      }
-
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-      }, maxDoc);
-  }
-
-  private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
-    final long valueCount = binaries.get(field.name).count;
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
-    final NumericEntry ordinalsEntry = ords.get(field.name);
-    final LongValues ordinals = getNumeric(ordinalsEntry);
-
-    final long[] table = ss.table;
-    final int[] offsets = ss.tableOffsets;
-
-    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
-
-      int offset, startOffset, endOffset;
-
-      @Override
-      public void setDocument(int docID) {
-        final int ord = (int) ordinals.get(docID);
-        offset = startOffset = offsets[ord];
-        endOffset = offsets[ord + 1];
-      }
-
-      @Override
-      public long nextOrd() {
-        if (offset == endOffset) {
-          return NO_MORE_ORDS;
-        } else {
-          return table[offset++];
-        }
-      }
-
-      @Override
-      public BytesRef lookupOrd(long ord) {
-        return binary.get(ord);
-      }
-
-      @Override
-      public long getValueCount() {
-        return valueCount;
-      }
-
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues) binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues) binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-      }, maxDoc);
-  }
-
-  private Bits getLiveBits(final long offset, final int count) throws IOException {
-    if (offset == ALL_MISSING) {
-      return new Bits.MatchNoBits(count);
-    } else if (offset == ALL_LIVE) {
-      return new Bits.MatchAllBits(count);
-    } else {
-      int length = (int) ((count + 7L) >>> 3);
-      final RandomAccessInput in = data.randomAccessSlice(offset, length);
-      return new Bits() {
-        @Override
-        public boolean get(int index) {
-          try {
-            return (in.readByte(index >> 3) & (1 << (index & 7))) != 0;
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-
-        @Override
-        public int length() {
-          return count;
-        }
-      };
-    }
-  }
-
-  private SparseNumericDocValues getSparseNumericDocValues(NumericEntry entry) throws IOException {
-    final RandomAccessInput docIdsData = this.data.randomAccessSlice(entry.missingOffset, entry.offset - entry.missingOffset);
-    final LongValues docIDs = DirectMonotonicReader.getInstance(entry.monotonicMeta, docIdsData);
-    final LongValues values = getNumeric(entry.nonMissingValues); // cannot be sparse
-    return new SparseNumericDocValues(Math.toIntExact(entry.numDocsWithValue), docIDs, values);
-  }
-
-  @Override
-  public synchronized DocValuesProducer getMergeInstance() throws IOException {
-    return new Lucene54DocValuesProducer(this);
-  }
-
-  @Override
-  public void close() throws IOException {
-    data.close();
-  }
-
-  /** metadata entry for a numeric docvalues field */
-  static class NumericEntry {
-    private NumericEntry() {}
-    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
-    long missingOffset;
-    /** offset to the actual numeric values */
-    public long offset;
-    /** end offset to the actual numeric values */
-    public long endOffset;
-    /** bits per value used to pack the numeric values */
-    public int bitsPerValue;
-
-    int format;
-    /** count of values written */
-    public long count;
-
-    /** monotonic meta */
-    public DirectMonotonicReader.Meta monotonicMeta;
-
-    long minValue;
-    long gcd;
-    long table[];
-
-    /** for sparse compression */
-    long numDocsWithValue;
-    NumericEntry nonMissingValues;
-    NumberType numberType;
-
-  }
-
-  /** metadata entry for a binary docvalues field */
-  static class BinaryEntry {
-    private BinaryEntry() {}
-    /** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
-    long missingOffset;
-    /** offset to the actual binary values */
-    long offset;
-
-    int format;
-    /** count of values written */
-    public long count;
-    int minLength;
-    int maxLength;
-    /** offset to the addressing data that maps a value to its slice of the byte[] */
-    public long addressesOffset, addressesEndOffset;
-    /** meta data for addresses */
-    public DirectMonotonicReader.Meta addressesMeta;
-    /** offset to the reverse index */
-    public long reverseIndexOffset;
-    /** packed ints version used to encode addressing information */
-    public int packedIntsVersion;
-    /** packed ints blocksize */
-    public int blockSize;
-  }
-
-  /** metadata entry for a sorted-set docvalues field */
-  static class SortedSetEntry {
-    private SortedSetEntry() {}
-    int format;
-
-    long[] table;
-    int[] tableOffsets;
-  }
-
-  // internally we compose complex dv (sorted/sortedset) from other ones
-  static abstract class LongBinaryDocValues extends LegacyBinaryDocValues {
-    @Override
-    public final BytesRef get(int docID) {
-      return get((long)docID);
-    }
-
-    abstract BytesRef get(long id);
-  }
-
-  // used for reverse lookup to a small range of blocks
-  static class ReverseTermsIndex implements Accountable {
-    public MonotonicBlockPackedReader termAddresses;
-    public PagedBytes.Reader terms;
-
-    @Override
-    public long ramBytesUsed() {
-      return termAddresses.ramBytesUsed() + terms.ramBytesUsed();
-    }
-
-    @Override
-    public Collection<Accountable> getChildResources() {
-      List<Accountable> resources = new ArrayList<>();
-      resources.add(Accountables.namedAccountable("term bytes", terms));
-      resources.add(Accountables.namedAccountable("term addresses", termAddresses));
-      return Collections.unmodifiableList(resources);
-    }
-
-    @Override
-    public String toString() {
-      return getClass().getSimpleName() + "(size=" + termAddresses.size() + ")";
-    }
-  }
-
-  //in the compressed case, we add a few additional operations for
-  //more efficient reverse lookup and enumeration
-  static final class CompressedBinaryDocValues extends LongBinaryDocValues {
-    final long numValues;
-    final long numIndexValues;
-    final int maxTermLength;
-    final MonotonicBlockPackedReader addresses;
-    final IndexInput data;
-    final CompressedBinaryTermsEnum termsEnum;
-    final PagedBytes.Reader reverseTerms;
-    final MonotonicBlockPackedReader reverseAddresses;
-    final long numReverseIndexValues;
-
-    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, ReverseTermsIndex index, IndexInput data) throws IOException {
-      this.maxTermLength = bytes.maxLength;
-      this.numValues = bytes.count;
-      this.addresses = addresses;
-      this.numIndexValues = addresses.size();
-      this.data = data;
-      this.reverseTerms = index.terms;
-      this.reverseAddresses = index.termAddresses;
-      this.numReverseIndexValues = reverseAddresses.size();
-      this.termsEnum = getTermsEnum(data);
-    }
-
-    @Override
-    public BytesRef get(long id) {
-      try {
-        termsEnum.seekExact(id);
-        return termsEnum.term();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    long lookupTerm(BytesRef key) {
-      try {
-        switch (termsEnum.seekCeil(key)) {
-          case FOUND: return termsEnum.ord();
-          case NOT_FOUND: return -termsEnum.ord()-1;
-          default: return -numValues-1;
-        }
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
-      }
-    }
-
-    TermsEnum getTermsEnum() {
-      try {
-        return getTermsEnum(data.clone());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    private CompressedBinaryTermsEnum getTermsEnum(IndexInput input) throws IOException {
-      return new CompressedBinaryTermsEnum(input);
-    }
-
-    class CompressedBinaryTermsEnum extends TermsEnum {
-      private long currentOrd = -1;
-      // offset to the start of the current block
-      private long currentBlockStart;
-      private final IndexInput input;
-      // delta from currentBlockStart to start of each term
-      private final int offsets[] = new int[INTERVAL_COUNT];
-      private final byte buffer[] = new byte[2*INTERVAL_COUNT-1];
-
-      private final BytesRef term = new BytesRef(maxTermLength);
-      private final BytesRef firstTerm = new BytesRef(maxTermLength);
-      private final BytesRef scratch = new BytesRef();
-
-      CompressedBinaryTermsEnum(IndexInput input) throws IOException {
-        this.input = input;
-        input.seek(0);
-      }
-
-      private void readHeader() throws IOException {
-        firstTerm.length = input.readVInt();
-        input.readBytes(firstTerm.bytes, 0, firstTerm.length);
-        input.readBytes(buffer, 0, INTERVAL_COUNT-1);
-        if (buffer[0] == -1) {
-          readShortAddresses();
-        } else {
-          readByteAddresses();
-        }
-        currentBlockStart = input.getFilePointer();
-      }
-
-      // read single byte addresses: each is delta - 2
-      // (shared prefix byte and length > 0 are both implicit)
-      private void readByteAddresses() throws IOException {
-        int addr = 0;
-        for (int i = 1; i < offsets.length; i++) {
-          addr += 2 + (buffer[i-1] & 0xFF);
-          offsets[i] = addr;
-        }
-      }
-
-      // read double byte addresses: each is delta - 2
-      // (shared prefix byte and length > 0 are both implicit)
-      private void readShortAddresses() throws IOException {
-        input.readBytes(buffer, INTERVAL_COUNT-1, INTERVAL_COUNT);
-        int addr = 0;
-        for (int i = 1; i < offsets.length; i++) {
-          int x = i<<1;
-          addr += 2 + ((buffer[x-1] << 8) | (buffer[x] & 0xFF));
-          offsets[i] = addr;
-        }
-      }
-
-      // set term to the first term
-      private void readFirstTerm() throws IOException {
-        term.length = firstTerm.length;
-        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, term.length);
-      }
-
-      // read term at offset, delta encoded from first term
-      private void readTerm(int offset) throws IOException {
-        int start = input.readByte() & 0xFF;
-        System.arraycopy(firstTerm.bytes, firstTerm.offset, term.bytes, 0, start);
-        int suffix = offsets[offset] - offsets[offset-1] - 1;
-        input.readBytes(term.bytes, start, suffix);
-        term.length = start + suffix;
-      }
-
-      @Override
-      public BytesRef next() throws IOException {
-        currentOrd++;
-        if (currentOrd >= numValues) {
-          return null;
-        } else {
-          int offset = (int) (currentOrd & INTERVAL_MASK);
-          if (offset == 0) {
-            // switch to next block
-            readHeader();
-            readFirstTerm();
-          } else {
-            readTerm(offset);
-          }
-          return term;
-        }
-      }
-
-      // binary search reverse index to find smaller
-      // range of blocks to search
-      long binarySearchIndex(BytesRef text) throws IOException {
-        long low = 0;
-        long high = numReverseIndexValues - 1;
-        while (low <= high) {
-          long mid = (low + high) >>> 1;
-          reverseTerms.fill(scratch, reverseAddresses.get(mid));
-          int cmp = scratch.compareTo(text);
-
-          if (cmp < 0) {
-            low = mid + 1;
-          } else if (cmp > 0) {
-            high = mid - 1;
-          } else {
-            return mid;
-          }
-        }
-        return high;
-      }
-
-      // binary search against first term in block range
-      // to find term's block
-      long binarySearchBlock(BytesRef text, long low, long high) throws IOException {
-        while (low <= high) {
-          long mid = (low + high) >>> 1;
-          input.seek(addresses.get(mid));
-          term.length = input.readVInt();
-          input.readBytes(term.bytes, 0, term.length);
-          int cmp = term.compareTo(text);
-
-          if (cmp < 0) {
-            low = mid + 1;
-          } else if (cmp > 0) {
-            high = mid - 1;
-          } else {
-            return mid;
-          }
-        }
-        return high;
-      }
-
-      @Override
-      public SeekStatus seekCeil(BytesRef text) throws IOException {
-        // locate block: narrow to block range with index, then search blocks
-        final long block;
-        long indexPos = binarySearchIndex(text);
-        if (indexPos < 0) {
-          block = 0;
-        } else {
-          long low = indexPos << BLOCK_INTERVAL_SHIFT;
-          long high = Math.min(numIndexValues - 1, low + BLOCK_INTERVAL_MASK);
-          block = Math.max(low, binarySearchBlock(text, low, high));
-        }
-
-        // position before block, then scan to term.
-        input.seek(addresses.get(block));
-        currentOrd = (block << INTERVAL_SHIFT) - 1;
-
-        while (next() != null) {
-          int cmp = term.compareTo(text);
-          if (cmp == 0) {
-            return SeekStatus.FOUND;
-          } else if (cmp > 0) {
-            return SeekStatus.NOT_FOUND;
-          }
-        }
-        return SeekStatus.END;
-      }
-
-      @Override
-      public void seekExact(long ord) throws IOException {
-        long block = ord >>> INTERVAL_SHIFT;
-        if (block != currentOrd >>> INTERVAL_SHIFT) {
-          // switch to different block
-          input.seek(addresses.get(block));
-          readHeader();
-        }
-
-        currentOrd = ord;
-
-        int offset = (int) (ord & INTERVAL_MASK);
-        if (offset == 0) {
-          readFirstTerm();
-        } else {
-          input.seek(currentBlockStart + offsets[offset-1]);
-          readTerm(offset);
-        }
-      }
-
-      @Override
-      public BytesRef term() throws IOException {
-        return term;
-      }
-
-      @Override
-      public long ord() throws IOException {
-        return currentOrd;
-      }
-
-      @Override
-      public int docFreq() throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public long totalTermFreq() throws IOException {
-        return -1;
-      }
-
-      @Override
-      public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-    }
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
deleted file mode 100644
index 5071075..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene62;
-
-import java.util.Objects;
-
-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.lucene50.Lucene50CompoundFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
-import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
-import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
-import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
-import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
-
-/**
- * Implements the Lucene 6.2 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.codecs.lucene60 package documentation for file format details.
- *
- * @lucene.experimental
- */
-public class Lucene62Codec extends Codec {
-  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
-  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
-  private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat();
-  private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
-  private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
-  
-  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
-    @Override
-    public PostingsFormat getPostingsFormatForField(String field) {
-      return Lucene62Codec.this.getPostingsFormatForField(field);
-    }
-  };
-  
-  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
-    @Override
-    public DocValuesFormat getDocValuesFormatForField(String field) {
-      return Lucene62Codec.this.getDocValuesFormatForField(field);
-    }
-  };
-  
-  private final StoredFieldsFormat storedFieldsFormat;
-
-  /** 
-   * Instantiates a new codec.
-   */
-  public Lucene62Codec() {
-    this(Mode.BEST_SPEED);
-  }
-  
-  /** 
-   * Instantiates a new codec, specifying the stored fields compression
-   * mode to use.
-   * @param mode stored fields compression mode to use for newly 
-   *             flushed/merged segments.
-   */
-  public Lucene62Codec(Mode mode) {
-    super("Lucene62");
-    this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
-  }
-  
-  @Override
-  public final StoredFieldsFormat storedFieldsFormat() {
-    return storedFieldsFormat;
-  }
-  
-  @Override
-  public final TermVectorsFormat termVectorsFormat() {
-    return vectorsFormat;
-  }
-
-  @Override
-  public final PostingsFormat postingsFormat() {
-    return postingsFormat;
-  }
-  
-  @Override
-  public final FieldInfosFormat fieldInfosFormat() {
-    return fieldInfosFormat;
-  }
-  
-  @Override
-  public final 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();
-  }
-
-  /** Returns the postings format that should be used for writing 
-   *  new segments of <code>field</code>.
-   *  
-   *  The default implementation always returns "Lucene50".
-   *  <p>
-   *  <b>WARNING:</b> if you subclass, you are responsible for index 
-   *  backwards compatibility: future version of Lucene are only 
-   *  guaranteed to be able to read the default implementation. 
-   */
-  public PostingsFormat getPostingsFormatForField(String field) {
-    return defaultFormat;
-  }
-  
-  /** Returns the docvalues format that should be used for writing 
-   *  new segments of <code>field</code>.
-   *  
-   *  The default implementation always returns "Lucene54".
-   *  <p>
-   *  <b>WARNING:</b> if you subclass, you are responsible for index 
-   *  backwards compatibility: future version of Lucene are only 
-   *  guaranteed to be able to read the default implementation. 
-   */
-  public DocValuesFormat getDocValuesFormatForField(String field) {
-    return defaultDVFormat;
-  }
-  
-  @Override
-  public final DocValuesFormat docValuesFormat() {
-    return docValuesFormat;
-  }
-
-  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
-  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene54");
-
-  private final NormsFormat normsFormat = new Lucene53NormsFormat();
-
-  @Override
-  public final NormsFormat normsFormat() {
-    return normsFormat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9128bdba/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
index 8d86649..7f9aed0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
@@ -35,7 +35,6 @@ import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
 import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
-import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
 import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
 import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
 import org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat;
@@ -168,7 +167,7 @@ public class Lucene70Codec extends Codec {
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
   private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
 
-  private final NormsFormat normsFormat = new Lucene53NormsFormat();
+  private final NormsFormat normsFormat = new Lucene70NormsFormat();
 
   @Override
   public final NormsFormat normsFormat() {