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

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

Repository: lucene-solr
Updated Branches:
  refs/heads/master 2ad00826a -> 32446e920


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
new file mode 100644
index 0000000..4c27f1e
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
@@ -0,0 +1,613 @@
+/*
+ * 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.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.lucene70.Lucene70DocValuesProducer.SparseNumericDocValues;
+import org.apache.lucene.codecs.lucene70.Lucene70DocValuesProducer.SparseNumericDocValuesRandomAccessWrapper;
+import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
+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 Lucene70DocValuesFormat
+ */
+public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene70DocValuesFormat());
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+  
+  // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
+  // for now just keep them here, as we want to test this for this format.
+  
+  @Slow
+  public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
+    }
+  }
+  
+  @Nightly
+  public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
+    }
+  }
+  
+  @Slow
+  public void testSortedVariableLengthBigVsStoredFields() throws Exception {
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedVsStoredFields(atLeast(300), 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 Lucene70DocValuesFormat();
+    conf.setCodec(new AssertingCodec() {
+      @Override
+      public PostingsFormat getPostingsFormatForField(String field) {
+        return pf;
+      }
+
+      @Override
+      public DocValuesFormat getDocValuesFormatForField(String field) {
+        return dv;
+      }
+    });
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    
+    // index some docs
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
+      doc.add(idField);
+      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, (int) 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 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+    for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
+      final Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+      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 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+    for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
+      final Directory dir = newDirectory();
+      IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
+      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/32446e92/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java b/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
index 892ab6c..aeef023 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
@@ -135,6 +135,6 @@ public class Test2BPoints extends LuceneTestCase {
   }
 
   private static Codec getCodec() {
-    return Codec.forName("Lucene62");
+    return Codec.forName("Lucene70");
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
index a846c27..a75290a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -393,11 +393,11 @@ public class TestPointValues extends LuceneTestCase {
     dir.close();
   }
 
-  // Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with SimpleText
+  // Write point values, one segment with Lucene70, another with SimpleText, then forceMerge with SimpleText
   public void testDifferentCodecs1() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(Codec.forName("Lucene62"));
+    iwc.setCodec(Codec.forName("Lucene70"));
     IndexWriter w = new IndexWriter(dir, iwc);
     Document doc = new Document();
     doc.add(new IntPoint("int", 1));
@@ -416,7 +416,7 @@ public class TestPointValues extends LuceneTestCase {
     dir.close();
   }
 
-  // Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with Lucene60
+  // Write point values, one segment with Lucene70, another with SimpleText, then forceMerge with Lucene70
   public void testDifferentCodecs2() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
@@ -428,7 +428,7 @@ public class TestPointValues extends LuceneTestCase {
     w.close();
     
     iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(Codec.forName("Lucene62"));
+    iwc.setCodec(Codec.forName("Lucene70"));
     w = new IndexWriter(dir, iwc);
     doc = new Document();
     doc.add(new IntPoint("int", 1));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
index 8bf49e7..0974563 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2.java
@@ -96,7 +96,7 @@ public class TestBoolean2 extends LuceneTestCase {
 
     IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
     // randomized codecs are sometimes too costly for this test:
-    iwc.setCodec(Codec.forName("Lucene62"));
+    iwc.setCodec(Codec.forName("Lucene70"));
     iwc.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter writer= new RandomIndexWriter(random(), directory, iwc);
     // we'll make a ton of docs, disable store/norms/vectors
@@ -141,7 +141,7 @@ public class TestBoolean2 extends LuceneTestCase {
     iwc = newIndexWriterConfig(new MockAnalyzer(random()));
     // we need docID order to be preserved:
     // randomized codecs are sometimes too costly for this test:
-    iwc.setCodec(Codec.forName("Lucene62"));
+    iwc.setCodec(Codec.forName("Lucene70"));
     iwc.setMergePolicy(newLogMergePolicy());
     try (IndexWriter w = new IndexWriter(singleSegmentDirectory, iwc)) {
       w.forceMerge(1, true);
@@ -167,7 +167,7 @@ public class TestBoolean2 extends LuceneTestCase {
 
       iwc = newIndexWriterConfig(new MockAnalyzer(random()));
       // randomized codecs are sometimes too costly for this test:
-      iwc.setCodec(Codec.forName("Lucene62"));
+      iwc.setCodec(Codec.forName("Lucene70"));
       RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
       w.addIndexes(copy);
       copy.close();
@@ -179,7 +179,7 @@ public class TestBoolean2 extends LuceneTestCase {
     iwc = newIndexWriterConfig(new MockAnalyzer(random()));
     iwc.setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000));
     // randomized codecs are sometimes too costly for this test:
-    iwc.setCodec(Codec.forName("Lucene62"));
+    iwc.setCodec(Codec.forName("Lucene70"));
     RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
 
     doc = new Document();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index af3d391..5ad71bf 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1157,14 +1157,14 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   private static Codec getCodec() {
-    if (Codec.getDefault().getName().equals("Lucene62")) {
+    if (Codec.getDefault().getName().equals("Lucene70")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
       double maxMBSortInHeap = 5.0 + (3*random().nextDouble());
       if (VERBOSE) {
         System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
       }
 
-      return new FilterCodec("Lucene62", Codec.getDefault()) {
+      return new FilterCodec("Lucene70", Codec.getDefault()) {
         @Override
         public PointsFormat pointsFormat() {
           return new PointsFormat() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
index 0b19254..2fb2077 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestNearest.java
@@ -247,7 +247,7 @@ public class TestNearest extends LuceneTestCase {
 
   private IndexWriterConfig getIndexWriterConfig() {
     IndexWriterConfig iwc = newIndexWriterConfig();
-    iwc.setCodec(Codec.forName("Lucene62"));
+    iwc.setCodec(Codec.forName("Lucene70"));
     return iwc;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
index c1aa6ee..97606c7 100644
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
@@ -85,14 +85,14 @@ import com.carrotsearch.randomizedtesting.generators.RandomInts;
 public class TestGeo3DPoint extends LuceneTestCase {
 
   private static Codec getCodec() {
-    if (Codec.getDefault().getName().equals("Lucene62")) {
+    if (Codec.getDefault().getName().equals("Lucene70")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
       double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
       if (VERBOSE) {
         System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
       }
 
-      return new FilterCodec("Lucene62", Codec.getDefault()) {
+      return new FilterCodec("Lucene70", Codec.getDefault()) {
         @Override
         public PointsFormat pointsFormat() {
           return new PointsFormat() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index 6b1c2d1..fe9992d 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -32,7 +32,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -646,7 +646,7 @@ public class TestSuggestField extends LuceneTestCase {
   static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set<String> suggestFields) {
     IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
     iwc.setMergePolicy(newLogMergePolicy());
-    Codec filterCodec = new Lucene62Codec() {
+    Codec filterCodec = new Lucene70Codec() {
       PostingsFormat postingsFormat = new Completion50PostingsFormat();
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
index 19b4a1c..d4896f8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
@@ -1239,7 +1239,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
     // Else seeds may not reproduce:
     iwc.setMergeScheduler(new SerialMergeScheduler());
     int pointsInLeaf = 2 + random().nextInt(4);
-    iwc.setCodec(new FilterCodec("Lucene62", TestUtil.getDefaultCodec()) {
+    iwc.setCodec(new FilterCodec("Lucene70", TestUtil.getDefaultCodec()) {
       @Override
       public PointsFormat pointsFormat() {
         return new PointsFormat() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
index 5c88dc7..279305e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
@@ -32,7 +32,7 @@ import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
 import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
 import org.apache.lucene.codecs.compressing.CompressingCodec;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
@@ -181,8 +181,8 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
       codec = new AssertingCodec();
     } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
       codec = CompressingCodec.randomInstance(random);
-    } else if ("Lucene62".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene62"))) {
-      codec = new Lucene62Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
+    } else if ("Lucene70".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene70"))) {
+      codec = new Lucene70Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
     } else if (!"random".equals(TEST_CODEC)) {
       codec = Codec.forName(TEST_CODEC);
     } else if ("random".equals(TEST_POSTINGSFORMAT)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 710e505..ee20584 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -53,8 +53,8 @@ import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
 import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
-import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
@@ -913,7 +913,7 @@ public final class TestUtil {
    * This may be different than {@link Codec#getDefault()} because that is randomized. 
    */
   public static Codec getDefaultCodec() {
-    return new Lucene62Codec();
+    return new Lucene70Codec();
   }
   
   /** 
@@ -946,7 +946,7 @@ public final class TestUtil {
    * Returns the actual default docvalues format (e.g. LuceneMNDocValuesFormat for this version of Lucene.
    */
   public static DocValuesFormat getDefaultDocValuesFormat() {
-    return new Lucene54DocValuesFormat();
+    return new Lucene70DocValuesFormat();
   }
 
   // TODO: generalize all 'test-checks-for-crazy-codecs' to

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
index 2b69d51..c69770c 100644
--- a/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
@@ -24,7 +24,7 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
@@ -91,7 +91,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
       compressionMode = SOLR_DEFAULT_COMPRESSION_MODE;
       log.debug("Using default compressionMode: " + compressionMode);
     }
-    codec = new Lucene62Codec(compressionMode) {
+    codec = new Lucene70Codec(compressionMode) {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
index 8cd0729..0454f3e 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
@@ -19,7 +19,7 @@
   <fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct"/>
   <fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene50"/>
 
-  <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene54"/>
+  <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene70"/>
   <fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory"/>
 
   <fieldType name="string" class="solr.StrField"/>


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

Posted by jp...@apache.org.
LUCENE-7453: Create a Lucene70Codec.


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

Branch: refs/heads/master
Commit: 32446e9205679fb94b247f0fa2aa97ecd54a49ff
Parents: 2ad0082
Author: Adrien Grand <jp...@gmail.com>
Authored: Mon Oct 3 17:11:07 2016 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Tue Oct 4 10:50:34 2016 +0200

----------------------------------------------------------------------
 .../benchmark/byTask/tasks/CreateIndexTask.java |    4 +-
 .../java/org/apache/lucene/codecs/Codec.java    |    2 +-
 .../lucene54/Lucene54DocValuesFormat.java       |    3 +
 .../lucene/codecs/lucene70/Lucene70Codec.java   |  177 ++
 .../lucene70/Lucene70DocValuesConsumer.java     |  797 ++++++++
 .../lucene70/Lucene70DocValuesFormat.java       |  183 ++
 .../lucene70/Lucene70DocValuesProducer.java     | 1816 ++++++++++++++++++
 .../lucene/codecs/lucene70/package-info.java    |  392 ++++
 .../services/org.apache.lucene.codecs.Codec     |    1 +
 .../org.apache.lucene.codecs.DocValuesFormat    |    1 +
 ...cene50StoredFieldsFormatHighCompression.java |    8 +-
 .../lucene53/TestLucene53NormsFormat.java       |    4 +-
 .../lucene70/TestLucene70DocValuesFormat.java   |  613 ++++++
 .../org/apache/lucene/index/Test2BPoints.java   |    2 +-
 .../apache/lucene/index/TestPointValues.java    |    8 +-
 .../org/apache/lucene/search/TestBoolean2.java  |    8 +-
 .../apache/lucene/search/TestPointQueries.java  |    4 +-
 .../org/apache/lucene/document/TestNearest.java |    2 +-
 .../apache/lucene/spatial3d/TestGeo3DPoint.java |    4 +-
 .../suggest/document/TestSuggestField.java      |    4 +-
 .../apache/lucene/geo/BaseGeoPointTestCase.java |    2 +-
 .../util/TestRuleSetupAndRestoreClassEnv.java   |    6 +-
 .../java/org/apache/lucene/util/TestUtil.java   |    8 +-
 .../apache/solr/core/SchemaCodecFactory.java    |    4 +-
 .../solr/collection1/conf/schema_codec.xml      |    2 +-
 25 files changed, 4019 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
----------------------------------------------------------------------
diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
index df8a1b4..c2c145b 100644
--- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
+++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java
@@ -29,7 +29,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene62.Lucene62Codec;
+import org.apache.lucene.codecs.lucene70.Lucene70Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
@@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene62Codec() {
+        iwConf.setCodec(new Lucene70Codec() {
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
index 442445c..d864710 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
@@ -57,7 +57,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
     }
     
     // TODO: should we use this, or maybe a system property is better?
-    static Codec defaultCodec = LOADER.lookup("Lucene62");
+    static Codec defaultCodec = LOADER.lookup("Lucene70");
   }
 
   private final String name;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/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
index 800649c..91ccfe2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesFormat.java
@@ -22,6 +22,7 @@ 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;
@@ -95,7 +96,9 @@ import org.apache.lucene.util.packed.DirectWriter;
  *   <li><tt>.dvm</tt>: DocValues metadata</li>
  * </ol>
  * @lucene.experimental
+ * @deprecated Use {@link Lucene70DocValuesFormat}.
  */
+@Deprecated
 public final class Lucene54DocValuesFormat extends DocValuesFormat {
 
   /** Sole Constructor */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/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
new file mode 100644
index 0000000..8d86649
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
@@ -0,0 +1,177 @@
+/*
+ * 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.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.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;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 7.0 index format, with configurable per-field postings
+ * and docvalues formats.
+ * <p>
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
+ * @see org.apache.lucene.codecs.lucene70 package documentation for file format details.
+ *
+ * @lucene.experimental
+ */
+public class Lucene70Codec extends Codec {
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new 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 Lucene70Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene70Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+  
+  private final StoredFieldsFormat storedFieldsFormat;
+
+  /** 
+   * Instantiates a new codec.
+   */
+  public Lucene70Codec() {
+    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 Lucene70Codec(Mode mode) {
+    super("Lucene70");
+    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 "Lucene70".
+   *  <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("Lucene70");
+
+  private final NormsFormat normsFormat = new Lucene53NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32446e92/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
new file mode 100644
index 0000000..564db7c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.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.lucene70;
+
+
+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.lucene70.Lucene70DocValuesFormat.*;
+
+/** writer for {@link Lucene70DocValuesFormat} */
+final class Lucene70DocValuesConsumer 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 Lucene70DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    boolean success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeIndexHeader(data, dataCodec, Lucene70DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeIndexHeader(meta, metaCodec, Lucene70DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      maxDoc = state.segmentInfo.maxDoc();
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+  
+  @Override
+  public void 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; // 10% of docs or less have a value
+    switch (numberType) {
+      case VALUE:
+        sparse = (double) missingCount / count >= 0.90;
+        break;
+      case ORDINAL:
+        sparse = (double) missingOrdCount / count >= 0.90;
+        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(Lucene70DocValuesFormat.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 {
+    long bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 64) {
+        data.writeLong(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1L << count;
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeLong(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(Lucene70DocValuesFormat.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(Lucene70DocValuesFormat.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(Lucene70DocValuesFormat.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(Lucene70DocValuesFormat.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(Lucene70DocValuesFormat.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(Lucene70DocValuesFormat.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(Lucene70DocValuesFormat.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/32446e92/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
new file mode 100644
index 0000000..2112341
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
@@ -0,0 +1,183 @@
+/*
+ * 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.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+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 7.0 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
+ */
+public final class Lucene70DocValuesFormat extends DocValuesFormat {
+
+  /** Sole Constructor */
+  public Lucene70DocValuesFormat() {
+    super("Lucene70");
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene70DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new Lucene70DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "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/3] lucene-solr:master: LUCENE-7453: Create a Lucene70Codec.

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

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

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

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

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

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