You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/06/11 15:49:24 UTC

[08/21] lucene-solr:branch_6x: LUCENE-6766: make new Lucene62Codec, with new segment info format that reads/writes the index sort, to support index sorting; remove all usage of SlowCompositeReaderWrapper; also sort ids when merging norms; CheckIndex veri

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java
new file mode 100644
index 0000000..53d2734
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene62;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter; // javadocs
+import org.apache.lucene.index.SegmentInfo; // javadocs
+import org.apache.lucene.index.SegmentInfos; // javadocs
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Version;
+
+// nocommit fixup javadocs here:
+
+/**
+ * Lucene 6.2 Segment info format.
+ * <p>
+ * Files:
+ * <ul>
+ *   <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Attributes, IndexSort, Footer
+ * </ul>
+ * Data types:
+ * <ul>
+ *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
+ *   <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
+ *   <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
+ *   <li>Files --&gt; {@link DataOutput#writeSetOfStrings Set&lt;String&gt;}</li>
+ *   <li>Diagnostics,Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
+ *   <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
+ *   <li>IndexSort --&gt; {@link DataOutput#writeInt Int32} count, followed by {@code count} SortField</li>
+ *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * Field Descriptions:
+ * <ul>
+ *   <li>SegVersion is the code version that created the segment.</li>
+ *   <li>SegSize is the number of documents contained in the segment index.</li>
+ *   <li>IsCompoundFile records whether the segment is written as a compound file or
+ *       not. If this is -1, the segment is not a compound file. If it is 1, the segment
+ *       is a compound file.</li>
+ *   <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
+ *       for each segment it creates. It includes metadata like the current Lucene
+ *       version, OS, Java version, why the segment was created (merge, flush,
+ *       addIndexes), etc.</li>
+ *   <li>Files is a list of files referred to by this segment.</li>
+ * </ul>
+ * 
+ * @see SegmentInfos
+ * @lucene.experimental
+ */
+public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
+
+  /** Sole constructor. */
+  public Lucene62SegmentInfoFormat() {
+  }
+  
+  @Override
+  public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene62SegmentInfoFormat.SI_EXTENSION);
+    try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      SegmentInfo si = null;
+      try {
+        int format = CodecUtil.checkIndexHeader(input, Lucene62SegmentInfoFormat.CODEC_NAME,
+                                                Lucene62SegmentInfoFormat.VERSION_START,
+                                                Lucene62SegmentInfoFormat.VERSION_CURRENT,
+                                                segmentID, "");
+        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+        
+        final int docCount = input.readInt();
+        if (docCount < 0) {
+          throw new CorruptIndexException("invalid docCount: " + docCount, input);
+        }
+        final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
+        
+        final Map<String,String> diagnostics = input.readMapOfStrings();
+        final Set<String> files = input.readSetOfStrings();
+        final Map<String,String> attributes = input.readMapOfStrings();
+
+        int numSortFields = input.readVInt();
+        Sort indexSort;
+        if (numSortFields > 0) {
+          SortField[] sortFields = new SortField[numSortFields];
+          for(int i=0;i<numSortFields;i++) {
+            String fieldName = input.readString();
+            int sortTypeID = input.readVInt();
+            SortField.Type sortType;
+            switch(sortTypeID) {
+            case 0:
+              sortType = SortField.Type.STRING;
+              break;
+            case 1:
+              sortType = SortField.Type.LONG;
+              break;
+            case 2:
+              sortType = SortField.Type.INT;
+              break;
+            default:
+              throw new CorruptIndexException("invalid index sort field type ID: " + sortTypeID, input);
+            }
+            byte b = input.readByte();
+            boolean reverse;
+            if (b == 0) {
+              reverse = true;
+            } else if (b == 1) {
+              reverse = false;
+            } else {
+              throw new CorruptIndexException("invalid index sort reverse: " + b, input);
+            }
+
+            sortFields[i] = new SortField(fieldName, sortType, reverse);
+
+            Object missingValue;
+            b = input.readByte();
+            if (b == 0) {
+              missingValue = null;
+            } else {
+              switch(sortType) {
+              case STRING:
+                if (b == 1) {
+                  missingValue = SortField.STRING_LAST;
+                } else if (b == 2) {
+                  missingValue = SortField.STRING_FIRST;
+                } else {
+                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
+                }
+                break;
+              case LONG:
+                if (b != 1) {
+                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
+                }
+                missingValue = input.readLong();
+                break;
+              case INT:
+                if (b != 1) {
+                  throw new CorruptIndexException("invalid missing value flag: " + b, input);
+                }
+                missingValue = input.readInt();
+                break;
+              default:
+                throw new AssertionError("unhandled sortType=" + sortType);
+              }
+            }
+            if (missingValue != null) {
+              sortFields[i].setMissingValue(missingValue);
+            }
+          }
+          indexSort = new Sort(sortFields);
+        } else if (numSortFields < 0) {
+          throw new CorruptIndexException("invalid index sort field count: " + numSortFields, input);
+        } else {
+          indexSort = null;
+        }
+        
+        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort);
+        si.setFiles(files);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+      return si;
+    }
+  }
+
+  @Override
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene62SegmentInfoFormat.SI_EXTENSION);
+
+    try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
+      // Only add the file once we've successfully created it, else IFD assert can trip:
+      si.addFile(fileName);
+      CodecUtil.writeIndexHeader(output, 
+                                   Lucene62SegmentInfoFormat.CODEC_NAME, 
+                                   Lucene62SegmentInfoFormat.VERSION_CURRENT,
+                                   si.getId(),
+                                   "");
+      Version version = si.getVersion();
+      if (version.major < 5) {
+        throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si);
+      }
+      // Write the Lucene version that created this segment, since 3.1
+      output.writeInt(version.major);
+      output.writeInt(version.minor);
+      output.writeInt(version.bugfix);
+      assert version.prerelease == 0;
+      output.writeInt(si.maxDoc());
+
+      output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+      output.writeMapOfStrings(si.getDiagnostics());
+      Set<String> files = si.files();
+      for (String file : files) {
+        if (!IndexFileNames.parseSegmentName(file).equals(si.name)) {
+          throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files);
+        }
+      }
+      output.writeSetOfStrings(files);
+      output.writeMapOfStrings(si.getAttributes());
+
+      Sort indexSort = si.getIndexSort();
+      int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
+      output.writeVInt(numSortFields);
+      for (int i = 0; i < numSortFields; ++i) {
+        SortField sortField = indexSort.getSort()[i];
+        output.writeString(sortField.getField());
+        int sortTypeID;
+        switch (sortField.getType()) {
+          case STRING:
+            sortTypeID = 0;
+            break;
+          case LONG:
+            sortTypeID = 1;
+            break;
+          case INT:
+            sortTypeID = 2;
+            break;
+          // nocommit the rest:
+          default:
+            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
+        }
+        output.writeVInt(sortTypeID);
+        output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
+
+        // write missing value 
+        Object missingValue = sortField.getMissingValue();
+        if (missingValue == null) {
+          output.writeByte((byte) 0);
+        } else {
+          switch(sortField.getType()) {
+          case STRING:
+            if (missingValue == SortField.STRING_LAST) {
+              output.writeByte((byte) 1);
+            } else if (missingValue == SortField.STRING_FIRST) {
+              output.writeByte((byte) 2);
+            } else {
+              throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue);
+            }
+            break;
+          case LONG:
+            output.writeByte((byte) 1);
+            output.writeLong(((Long) missingValue).longValue());
+            break;
+          case INT:
+            output.writeByte((byte) 1);
+            output.writeInt(((Integer) missingValue).intValue());
+            break;
+          // nocommit the rest:
+          default:
+            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
+          }
+        }
+      }
+      
+      CodecUtil.writeFooter(output);
+    }
+  }
+
+  /** File extension used to store {@link SegmentInfo}. */
+  public final static String SI_EXTENSION = "si";
+  static final String CODEC_NAME = "Lucene62SegmentInfo";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+}

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 5a68a3d..fb2dc80 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -43,6 +43,9 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
@@ -217,6 +220,9 @@ public final class CheckIndex implements Closeable {
 
       /** Status for testing of PointValues (null if PointValues could not be tested). */
       public PointsStatus pointsStatus;
+
+      /** Status of index sort */
+      public IndexSortStatus indexSortStatus;
     }
     
     /**
@@ -374,6 +380,16 @@ public final class CheckIndex implements Closeable {
       /** Exception thrown during doc values test (null on success) */
       public Throwable error = null;
     }
+
+    /**
+     * Status from testing index sort
+     */
+    public static final class IndexSortStatus {
+
+      /** Exception thrown during term index test (null on success) */
+      public Throwable error = null;
+    }
+    
   }
 
   /** Create a new CheckIndex on the directory. */
@@ -642,6 +658,10 @@ public final class CheckIndex implements Closeable {
         msg(infoStream, "    compound=" + info.info.getUseCompoundFile());
         segInfoStat.compound = info.info.getUseCompoundFile();
         msg(infoStream, "    numFiles=" + info.files().size());
+        Sort indexSort = info.info.getIndexSort();
+        if (indexSort != null) {
+          msg(infoStream, "    sort=" + indexSort);
+        }
         segInfoStat.numFiles = info.files().size();
         segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
         msg(infoStream, "    size (MB)=" + nf.format(segInfoStat.sizeMB));
@@ -722,6 +742,9 @@ public final class CheckIndex implements Closeable {
           // Test PointValues
           segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
 
+          // Test index sort
+          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
           if (segInfoStat.liveDocStatus.error != null) {
@@ -790,6 +813,70 @@ public final class CheckIndex implements Closeable {
 
     return result;
   }
+
+  public static Status.IndexSortStatus testSort(CodecReader reader, Sort sort, PrintStream infoStream, boolean failFast) throws IOException {
+    // This segment claims its documents are sorted according to the incoming sort ... let's make sure:
+
+    long startNS = System.nanoTime();
+
+    Status.IndexSortStatus status = new Status.IndexSortStatus();
+
+    if (sort != null) {
+      if (infoStream != null) {
+        infoStream.print("    test: check index sort.....");
+      }
+
+      SortField fields[] = sort.getSort();
+      final int reverseMul[] = new int[fields.length];
+      final LeafFieldComparator comparators[] = new LeafFieldComparator[fields.length];
+
+      LeafReaderContext readerContext = new LeafReaderContext(reader);
+    
+      for (int i = 0; i < fields.length; i++) {
+        reverseMul[i] = fields[i].getReverse() ? -1 : 1;
+        comparators[i] = fields[i].getComparator(1, i).getLeafComparator(readerContext);
+        // nocommit we prevent SCORE?
+        //comparators[i].setScorer(FAKESCORER);
+      }
+
+      int maxDoc = reader.maxDoc();
+
+      try {
+
+        for(int docID=1;docID < maxDoc;docID++) {
+      
+          int cmp = 0;
+
+          for (int i = 0; i < comparators.length; i++) {
+            // TODO: would be better if copy() didnt cause a term lookup in TermOrdVal & co,
+            // the segments are always the same here...
+            comparators[i].copy(0, docID-1);
+            comparators[i].setBottom(0);
+            cmp = reverseMul[i] * comparators[i].compareBottom(docID);
+            if (cmp != 0) {
+              break;
+            }
+          }
+
+          if (cmp > 0) {
+            throw new RuntimeException("segment has indexSort=" + sort + " but docID=" + (docID-1) + " sorts after docID=" + docID);
+          }
+        }
+        msg(infoStream, String.format(Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime()-startNS)));
+      } catch (Throwable e) {
+        if (failFast) {
+          IOUtils.reThrow(e);
+        }
+        msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
+        status.error = e;
+        if (infoStream != null) {
+          e.printStackTrace(infoStream);
+        }
+      }
+    }
+
+    return status;
+  }
   
   /**
    * Test live docs.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java b/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
index c4333bc..41c2a46 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
@@ -62,7 +62,6 @@ final class MappingMultiPostingsEnum extends PostingsEnum {
     this.field = field;
     allSubs = new MappingPostingsSub[mergeState.fieldsProducers.length];
     for(int i=0;i<allSubs.length;i++) {
-      // nocommit delDocMaps?
       allSubs[i] = new MappingPostingsSub(mergeState.docMaps[i], mergeState.liveDocs[i]);
     }
     this.docIDMerger = new DocIDMerger<MappingPostingsSub>(subs, allSubs.length, mergeState.segmentInfo.getIndexSort() != null);
@@ -89,7 +88,11 @@ final class MappingMultiPostingsEnum extends PostingsEnum {
 
   @Override
   public int docID() {
-    return current.mappedDocID;
+    if (current == null) {
+      return -1;
+    } else {
+      return current.mappedDocID;
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/index/Sorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/Sorter.java b/lucene/core/src/java/org/apache/lucene/index/Sorter.java
index 0ce7d64..cf75c18 100644
--- a/lucene/core/src/java/org/apache/lucene/index/Sorter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/Sorter.java
@@ -33,7 +33,6 @@ import org.apache.lucene.util.packed.PackedLongValues;
  * IDs.
  * @lucene.experimental
  */
-// nocommit rename to IndexSorter?
 final class Sorter {
   final Sort sort;
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
index 45d4482..b6558f7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
@@ -337,10 +337,7 @@ class SortingLeafReader extends FilterLeafReader {
 
     @Override
     public long nextOrd() {
-      // nocommit
-      long v = in.nextOrd();
-      //System.out.println("  slr.sssdv.nextOrd return " + v + " this=" + this);
-      return v;
+      return in.nextOrd();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 875aba5..548f8d0 100644
--- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene60.Lucene60Codec
+org.apache.lucene.codecs.lucene62.Lucene62Codec

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 59e4814..f945c2d 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.lucene60.Lucene60Codec;
+import org.apache.lucene.codecs.lucene62.Lucene62Codec;
 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 Lucene60Codec(Mode.BEST_COMPRESSION);
+    return new Lucene62Codec(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 Lucene60Codec(RandomPicks.randomFrom(random(), Mode.values())));
+      iwc.setCodec(new Lucene62Codec(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 Lucene60Codec(null);
+      new Lucene62Codec(null);
     });
     
     expectThrows(NullPointerException.class, () -> {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 c915de0..a0ad87f 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.lucene60.Lucene60Codec;
+import org.apache.lucene.codecs.lucene62.Lucene62Codec;
 import org.apache.lucene.index.BaseNormsFormatTestCase;
 
 /**
  * Tests Lucene53NormsFormat
  */
 public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
-  private final Codec codec = new Lucene60Codec();
+  private final Codec codec = new Lucene62Codec();
   
   @Override
   protected Codec getCodec() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java
new file mode 100644
index 0000000..8c758f2
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java
@@ -0,0 +1,39 @@
+package org.apache.lucene.codecs.lucene62;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.Version;
+
+/**
+ * Tests Lucene62SegmentInfoFormat
+ */
+public class TestLucene62SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
+
+  @Override
+  protected Version[] getVersions() {
+    return new Version[] { Version.LATEST };
+  }
+
+  @Override
+  protected Codec getCodec() {
+    return TestUtil.getDefaultCodec();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 2f3a3a6..da8dbac 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
@@ -24,8 +24,6 @@ import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PointsWriter;
-import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
-import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.search.IndexSearcher;
@@ -143,6 +141,6 @@ public class Test2BPoints extends LuceneTestCase {
   }
 
   private static Codec getCodec() {
-    return Codec.forName("Lucene60");
+    return Codec.forName("Lucene62");
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java b/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
index 22b3605..22d1234 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BTerms.java
@@ -53,7 +53,7 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
 // disk (but, should run successfully).  Best to run w/
 // -Dtests.codec=<current codec>, and w/ plenty of RAM, eg:
 //
-//   ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene60 -Dtestcase=Test2BTerms
+//   ant test -Dtests.monster=true -Dtests.heapsize=8g -Dtests.codec=Lucene62 -Dtestcase=Test2BTerms
 //
 @SuppressCodecs({ "SimpleText", "Memory", "Direct" })
 @Monster("very slow, use 5g minimum heap")

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index 2635b00..8df81ba 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -33,7 +33,6 @@ import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.BinaryPoint;
 import org.apache.lucene.document.Document;
@@ -76,16 +75,17 @@ import org.junit.BeforeClass;
 
 // nocommit test EarlyTerminatingCollector
 
+// nocommit must test all supported SortField.Type
+
 public class TestIndexSorting extends LuceneTestCase {
 
   public void testSortOnMerge(boolean withDeletes) throws IOException {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
     Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
     iwc.setIndexSort(indexSort);
     IndexWriter w = new IndexWriter(dir, iwc);
-    final int numDocs = atLeast(200);
+    final int numDocs = atLeast(1000);
     final FixedBitSet deleted = new FixedBitSet(numDocs);
     for (int i = 0; i < numDocs; ++i) {
       Document doc = new Document();
@@ -212,7 +212,6 @@ public class TestIndexSorting extends LuceneTestCase {
   public void testConcurrentUpdates() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
     Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
     iwc.setIndexSort(indexSort);
     IndexWriter w = new IndexWriter(dir, iwc);
@@ -303,7 +302,6 @@ public class TestIndexSorting extends LuceneTestCase {
   public void testConcurrentDVUpdates() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
     Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
     iwc.setIndexSort(indexSort);
     IndexWriter w = new IndexWriter(dir, iwc);
@@ -314,6 +312,7 @@ public class TestIndexSorting extends LuceneTestCase {
       Document doc = new Document();
       doc.add(new StringField("id", Integer.toString(i), Store.NO));
       doc.add(new NumericDocValuesField("foo", -1));
+      w.addDocument(doc);
       values.put(i, -1L);
     }
     Thread[] threads = new Thread[2];
@@ -321,7 +320,7 @@ public class TestIndexSorting extends LuceneTestCase {
     final CountDownLatch latch = new CountDownLatch(1);
     for (int i = 0; i < threads.length; ++i) {
       Random r = new Random(random().nextLong());
-      threads[i] = new Thread(new UpdateRunnable(numDocs, r, latch, updateCount, w, values));
+      threads[i] = new Thread(new DVUpdateRunnable(numDocs, r, latch, updateCount, w, values));
     }
     for (Thread thread : threads) {
       thread.start();
@@ -362,7 +361,6 @@ public class TestIndexSorting extends LuceneTestCase {
 
     Directory dir2 = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
     Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
     iwc.setIndexSort(indexSort);
     IndexWriter w2 = new IndexWriter(dir2, iwc);
@@ -410,7 +408,6 @@ public class TestIndexSorting extends LuceneTestCase {
   public void testIllegalChangeSort() throws Exception {
     final Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
     iwc.setIndexSort(new Sort(new SortField("foo", SortField.Type.LONG)));
     IndexWriter w = new IndexWriter(dir, iwc);
     w.addDocument(new Document());
@@ -420,12 +417,13 @@ public class TestIndexSorting extends LuceneTestCase {
     w.close();
 
     final IndexWriterConfig iwc2 = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc2.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
     iwc2.setIndexSort(new Sort(new SortField("bar", SortField.Type.LONG)));
-    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+    IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
         new IndexWriter(dir, iwc2);
     });
-    assertEquals("cannot change previous indexSort=<long: \"foo\"> (from segment=_2(7.0.0):c2:[indexSort=<long: \"foo\">]) to new indexSort=<long: \"bar\">", expected.getMessage());
+    String message = e.getMessage();
+    assertTrue(message.contains("cannot change previous indexSort=<long: \"foo\">"));
+    assertTrue(message.contains("to new indexSort=<long: \"bar\">"));
     dir.close();
   }
 
@@ -574,8 +572,6 @@ public class TestIndexSorting extends LuceneTestCase {
     
     PositionsTokenStream positions = new PositionsTokenStream();
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
-    // nocommit:
-    conf.setCodec(new SimpleTextCodec());
     conf.setMaxBufferedDocs(4); // create some segments
     conf.setSimilarity(new NormsSimilarity(conf.getSimilarity())); // for testing norms field
     // nocommit
@@ -633,7 +629,6 @@ public class TestIndexSorting extends LuceneTestCase {
       assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
       PostingsEnum sortedPositions = termsEnum.postings(null, PostingsEnum.ALL);
       int doc;
-      boolean isSorted = reader.getIndexSort() != null;
     
       // test nextDoc()
       while ((doc = sortedPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
@@ -687,7 +682,6 @@ public class TestIndexSorting extends LuceneTestCase {
       LeafReader reader = ctx.reader();
       NumericDocValues dv = reader.getNormValues(NORMS_FIELD);
       int maxDoc = reader.maxDoc();
-      boolean isSorted = reader.getIndexSort() != null;
       for (int doc = 0; doc < maxDoc; doc++) {
         int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
         assertEquals("incorrect norm value for doc " + doc, id, dv.get(doc));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 9f55ec3..9693c5c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -394,11 +394,11 @@ public class TestPointValues extends LuceneTestCase {
     dir.close();
   }
 
-  // Write point values, one segment with Lucene60, another with SimpleText, then forceMerge with SimpleText
+  // Write point values, one segment with Lucene62, 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("Lucene60"));
+    iwc.setCodec(Codec.forName("Lucene62"));
     IndexWriter w = new IndexWriter(dir, iwc);
     Document doc = new Document();
     doc.add(new IntPoint("int", 1));
@@ -417,7 +417,7 @@ public class TestPointValues extends LuceneTestCase {
     dir.close();
   }
 
-  // Write point values, one segment with Lucene60, another with SimpleText, then forceMerge with Lucene60
+  // Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with Lucene60
   public void testDifferentCodecs2() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
@@ -429,7 +429,7 @@ public class TestPointValues extends LuceneTestCase {
     w.close();
     
     iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(Codec.forName("Lucene60"));
+    iwc.setCodec(Codec.forName("Lucene62"));
     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/1530c5a9/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 df89e8b..cf8372d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -1154,14 +1154,14 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   private static Codec getCodec() {
-    if (Codec.getDefault().getName().equals("Lucene60")) {
+    if (Codec.getDefault().getName().equals("Lucene62")) {
       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("Lucene60", Codec.getDefault()) {
+      return new FilterCodec("Lucene62", Codec.getDefault()) {
         @Override
         public PointsFormat pointsFormat() {
           return new PointsFormat() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
index 4d76fa9..55f360a 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
@@ -21,7 +21,6 @@ import java.util.Collections;
 import java.util.Iterator;
 
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
@@ -29,11 +28,13 @@ import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.Sort;
 import org.apache.lucene.util.Bits;
 
 /**
@@ -178,4 +179,8 @@ public class TermVectorLeafReader extends LeafReader {
   public void document(int docID, StoredFieldVisitor visitor) throws IOException {
   }
 
+  @Override
+  public Sort getIndexSort() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index 5b13301..e3aa4b1 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -40,6 +40,7 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.*;
@@ -1606,6 +1607,10 @@ public class MemoryIndex {
       return info.getNormDocValues();
     }
 
+    @Override
+    public Sort getIndexSort() {
+      return null;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
index cf860a2..54ad744 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
@@ -35,7 +35,6 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
index 3bcc492..6108992 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
@@ -50,8 +50,6 @@ import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.uninverting.UninvertingReader.Type;
-import org.apache.lucene.uninverting.UninvertingReader;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -236,27 +234,12 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
     }
   }
 
-  private IndexSearcher newSearcherForTestTerminatedEarly(IndexReader r) throws IOException {
-    switch(random().nextInt(2)) {
-    case 0:
-      return new IndexSearcher(r);
-    case 1:
-      assertTrue(r+" is not a DirectoryReader", (r instanceof DirectoryReader));
-      final DirectoryReader directoryReader = ExitableDirectoryReader.wrap(
-          UninvertingReader.wrap((DirectoryReader) r, new HashMap<String,Type>()),
-          new TestEarlyTerminatingSortingcollectorQueryTimeout(false));
-      return new IndexSearcher(directoryReader);
-    }
-    fail("newSearcherForTestTerminatedEarly("+r+") fell through switch");
-    return null;
-  }
-
   public void testTerminatedEarly() throws IOException {
     final int iters = atLeast(8);
     for (int i = 0; i < iters; ++i) {
       createRandomIndex(true);
 
-      final IndexSearcher searcher = newSearcherForTestTerminatedEarly(reader); // future TODO: use newSearcher(reader);
+      final IndexSearcher searcher = new IndexSearcher(reader); // future TODO: use newSearcher(reader);
       final Query query = new MatchAllDocsQuery(); // search for everything/anything
 
       final TestTerminatedEarlySimpleCollector collector1 = new TestTerminatedEarlySimpleCollector();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 66630df..0b19254 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("Lucene60"));
+    iwc.setCodec(Codec.forName("Lucene62"));
     return iwc;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 5cc6e35..fed5ada 100644
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
@@ -86,14 +86,14 @@ import com.carrotsearch.randomizedtesting.generators.RandomInts;
 public class TestGeo3DPoint extends LuceneTestCase {
 
   private static Codec getCodec() {
-    if (Codec.getDefault().getName().equals("Lucene60")) {
+    if (Codec.getDefault().getName().equals("Lucene62")) {
       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("Lucene60", Codec.getDefault()) {
+      return new FilterCodec("Lucene62", Codec.getDefault()) {
         @Override
         public PointsFormat pointsFormat() {
           return new PointsFormat() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index 2b14d6e..004aef4 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -56,7 +56,6 @@ import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.SortingMergePolicy;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanClause;
@@ -232,7 +231,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     // This way all merged segments will be sorted at
     // merge time, allow for per-segment early termination
     // when those segments are searched:
-    iwc.setMergePolicy(new SortingMergePolicy(iwc.getMergePolicy(), SORT));
+    iwc.setIndexSort(SORT);
 
     return iwc;
   }
@@ -586,8 +585,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
 
     // We sorted postings by weight during indexing, so we
     // only retrieve the first num hits now:
-    final SortingMergePolicy sortingMergePolicy = (SortingMergePolicy) writer.getConfig().getMergePolicy();
-    Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num, sortingMergePolicy.getSort());
+    Collector c2 = new EarlyTerminatingSortingCollector(c, SORT, num);
     IndexSearcher searcher = searcherMgr.acquire();
     List<LookupResult> results = null;
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 62ed08b..6b1c2d1 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.lucene60.Lucene60Codec;
+import org.apache.lucene.codecs.lucene62.Lucene62Codec;
 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 Lucene60Codec() {
+    Codec filterCodec = new Lucene62Codec() {
       PostingsFormat postingsFormat = new Completion50PostingsFormat();
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 bda4cde..275c186 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
@@ -1242,7 +1242,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("Lucene60", TestUtil.getDefaultCodec()) {
+    iwc.setCodec(new FilterCodec("Lucene62", TestUtil.getDefaultCodec()) {
       @Override
       public PointsFormat pointsFormat() {
         return new PointsFormat() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
index 27e01c1..49d19ae 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
@@ -163,8 +163,14 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     }
   }
 
+  protected boolean supportsIndexSort() {
+    return true;
+  }
+
   /** Test sort */
   public void testSort() throws IOException {
+    assumeTrue("test requires a codec that can read/write index sort", supportsIndexSort());
+
     final int iters = atLeast(5);
     for (int i = 0; i < iters; ++i) {
       Sort sort;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1530c5a9/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 7ac4037..5c88dc7 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.lucene60.Lucene60Codec;
+import org.apache.lucene.codecs.lucene62.Lucene62Codec;
 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 ("Lucene60".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene60"))) {
-      codec = new Lucene60Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
+    } 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 (!"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/1530c5a9/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 d772ae3..b632160 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
@@ -54,7 +54,7 @@ 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.lucene60.Lucene60Codec;
+import org.apache.lucene.codecs.lucene62.Lucene62Codec;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
@@ -911,7 +911,7 @@ public final class TestUtil {
    * This may be different than {@link Codec#getDefault()} because that is randomized. 
    */
   public static Codec getDefaultCodec() {
-    return new Lucene60Codec();
+    return new Lucene62Codec();
   }
   
   /**