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:08:00 UTC

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

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;