You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by so...@apache.org on 2020/06/03 19:13:48 UTC

[lucene-solr] 19/47: LUCENE-9330: Make SortFields responsible for index sorting and serialization (#1440)

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

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

commit 1ef077f5661b2092dd718d3022c38084cce8440b
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Fri May 22 13:33:06 2020 +0100

    LUCENE-9330: Make SortFields responsible for index sorting and serialization (#1440)
    
    This commit adds a new class IndexSorter which handles how a sort should be applied
    to documents in an index:
    
    * how to serialize/deserialize sort info in the segment header
    * how to sort documents within a segment
    * how to sort documents from merging segments
    
    SortField has a getIndexSorter() method, which will return null if the sort cannot be used
    to sort an index (eg if it uses scores or other query-dependent values). This also requires a
    new Codec as there is a change to the SegmentInfoFormat
---
 lucene/CHANGES.txt                                 |   3 +
 .../codecs/lucene70/Lucene70SegmentInfoFormat.java | 168 +-------
 .../lucene/codecs/lucene70/package-info.java       |   2 +-
 .../lucene/codecs/lucene84/Lucene84Codec.java      |  46 +--
 .../lucene/codecs/lucene84}/package-info.java      |   4 +-
 .../services/org.apache.lucene.codecs.Codec        |   1 +
 .../lucene70/Lucene70RWSegmentInfoFormat.java      | 204 ++++++++++
 .../lucene70/TestLucene70SegmentInfoFormat.java    |  13 +-
 .../benchmark/byTask/tasks/CreateIndexTask.java    |   4 +-
 .../simpletext/SimpleTextSegmentInfoFormat.java    | 306 ++------------
 .../src/java/org/apache/lucene/codecs/Codec.java   |   2 +-
 .../lucene/codecs/lucene84/package-info.java       | 396 +-----------------
 .../Lucene86Codec.java}                            |  72 ++--
 .../codecs/lucene86/Lucene86SegmentInfoFormat.java | 217 ++++++++++
 .../{lucene84 => lucene86}/package-info.java       |  60 +--
 .../apache/lucene/index/BinaryDocValuesWriter.java |  29 +-
 .../apache/lucene/index/DefaultIndexingChain.java  | 183 ++++++---
 .../apache/lucene/index/DocValuesLeafReader.java   |  89 ++++
 .../org/apache/lucene/index/DocValuesWriter.java   |   8 +-
 .../java/org/apache/lucene/index/IndexSorter.java  | 448 +++++++++++++++++++++
 .../org/apache/lucene/index/IndexWriterConfig.java |  17 +-
 .../java/org/apache/lucene/index/MultiSorter.java  | 144 +------
 .../lucene/index/NumericDocValuesWriter.java       |  33 +-
 .../org/apache/lucene/index/SortFieldProvider.java | 118 ++++++
 .../apache/lucene/index/SortedDocValuesWriter.java |  50 +--
 .../lucene/index/SortedNumericDocValuesWriter.java |  33 +-
 .../lucene/index/SortedSetDocValuesWriter.java     |  45 +--
 .../src/java/org/apache/lucene/index/Sorter.java   | 238 +----------
 .../java/org/apache/lucene/search/SortField.java   | 135 +++++++
 .../lucene/search/SortedNumericSortField.java      | 106 +++++
 .../apache/lucene/search/SortedSetSortField.java   |  69 +++-
 .../services/org.apache.lucene.codecs.Codec        |   2 +-
 .../org.apache.lucene.index.SortFieldProvider      |  20 +
 ...tLucene50StoredFieldsFormatHighCompression.java |  11 +-
 .../codecs/lucene80/TestLucene80NormsFormat.java   |   4 +-
 .../TestLucene86SegmentInfoFormat.java}            |   6 +-
 .../org/apache/lucene/index/TestIndexSorting.java  |   5 +-
 .../org/apache/lucene/index/TestPointValues.java   |   4 +-
 .../org/apache/lucene/search/TestBoolean2.java     |   8 +-
 .../document/TestFloatPointNearestNeighbor.java    |   2 +-
 .../test/org/apache/lucene/search/TestNearest.java |   2 +-
 .../search/suggest/document/TestSuggestField.java  |  11 +-
 .../apache/lucene/geo/BaseGeoPointTestCase.java    |   2 +-
 .../org/apache/lucene/geo/BaseXYPointTestCase.java |   2 +-
 .../util/TestRuleSetupAndRestoreClassEnv.java      |  29 +-
 .../src/java/org/apache/lucene/util/TestUtil.java  |   4 +-
 .../org/apache/solr/core/SchemaCodecFactory.java   |   4 +-
 47 files changed, 1832 insertions(+), 1527 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 46c7063..078e8b5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -149,6 +149,9 @@ API Changes
 * LUCENE-9339: MergeScheduler#merge doesn't accept a parameter if a new merge was found anymore.
   (Simon Willnauer)
 
+* LUCENE-9330: SortFields are now responsible for writing themselves into index headers if they
+  are used as index sorts.  (Alan Woodward, Uwe Schindler, Adrien Grand)
+
 * LUCENE-9340: Deprecate SimpleBindings#add(SortField). (Alan Woodward)
 
 * LUCENE-9345: MergeScheduler is now decoupled from IndexWriter. Instead it accepts a MergeSource
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java
similarity index 63%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java
rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java
index ed55770..ab54012 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java
@@ -24,9 +24,9 @@ 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.index.IndexWriter;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortedNumericSelector;
@@ -34,10 +34,9 @@ import org.apache.lucene.search.SortedNumericSortField;
 import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.search.SortedSetSortField;
 import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.Version;
 
 /**
@@ -271,164 +270,7 @@ public class Lucene70SegmentInfoFormat extends SegmentInfoFormat {
 
   @Override
   public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene70SegmentInfoFormat.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,
-                                   Lucene70SegmentInfoFormat.CODEC_NAME,
-                                   Lucene70SegmentInfoFormat.VERSION_CURRENT,
-                                   si.getId(),
-                                   "");
-      Version version = si.getVersion();
-      if (version.major < 7) {
-        throw new IllegalArgumentException("invalid major version: should be >= 7 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);
-
-      // Write the min Lucene version that contributed docs to the segment, since 7.0
-      if (si.getMinVersion() != null) {
-        output.writeByte((byte) 1);
-        Version minVersion = si.getMinVersion();
-        output.writeInt(minVersion.major);
-        output.writeInt(minVersion.minor);
-        output.writeInt(minVersion.bugfix);
-      } else {
-        output.writeByte((byte) 0);
-      }
-
-      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];
-        SortField.Type sortType = sortField.getType();
-        output.writeString(sortField.getField());
-        int sortTypeID;
-        switch (sortField.getType()) {
-          case STRING:
-            sortTypeID = 0;
-            break;
-          case LONG:
-            sortTypeID = 1;
-            break;
-          case INT:
-            sortTypeID = 2;
-            break;
-          case DOUBLE:
-            sortTypeID = 3;
-            break;
-          case FLOAT:
-            sortTypeID = 4;
-            break;
-          case CUSTOM:
-            if (sortField instanceof SortedSetSortField) {
-              sortTypeID = 5;
-              sortType = SortField.Type.STRING;
-            } else if (sortField instanceof SortedNumericSortField) {
-              sortTypeID = 6;
-              sortType = ((SortedNumericSortField) sortField).getNumericType();
-            } else {
-              throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField);
-            }
-            break;
-          default:
-            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
-        }
-        output.writeVInt(sortTypeID);
-        if (sortTypeID == 5) {
-          SortedSetSortField ssf = (SortedSetSortField) sortField;
-          if (ssf.getSelector() == SortedSetSelector.Type.MIN) {
-            output.writeByte((byte) 0);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) {
-            output.writeByte((byte) 1);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) {
-            output.writeByte((byte) 2);
-          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) {
-            output.writeByte((byte) 3);
-          } else {
-            throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector());
-          }
-        } else if (sortTypeID == 6) {
-          SortedNumericSortField snsf = (SortedNumericSortField) sortField;
-          if (snsf.getNumericType() == SortField.Type.LONG) {
-            output.writeByte((byte) 0);
-          } else if (snsf.getNumericType() == SortField.Type.INT) {
-            output.writeByte((byte) 1);
-          } else if (snsf.getNumericType() == SortField.Type.DOUBLE) {
-            output.writeByte((byte) 2);
-          } else if (snsf.getNumericType() == SortField.Type.FLOAT) {
-            output.writeByte((byte) 3);
-          } else {
-            throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType());
-          }
-          if (snsf.getSelector() == SortedNumericSelector.Type.MIN) {
-            output.writeByte((byte) 0);
-          } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) {
-            output.writeByte((byte) 1);
-          } else {
-            throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector());
-          }
-        }
-        output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
-
-        // write missing value 
-        Object missingValue = sortField.getMissingValue();
-        if (missingValue == null) {
-          output.writeByte((byte) 0);
-        } else {
-          switch(sortType) {
-          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;
-          case DOUBLE:
-            output.writeByte((byte) 1);
-            output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue()));
-            break;
-          case FLOAT:
-            output.writeByte((byte) 1);
-            output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue()));
-            break;
-          default:
-            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
-          }
-        }
-      }
-
-      CodecUtil.writeFooter(output);
-    }
+    throw new UnsupportedOperationException("Old formats can't be used for writing");
   }
 
   /** File extension used to store {@link SegmentInfo}. */
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/package-info.java
similarity index 96%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
copy to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/package-info.java
index e1913a0..6bbf70c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/package-info.java
@@ -16,7 +16,7 @@
  */
 
 /**
- * Components from the Lucene 7.0 index format.  See {@link org.apache.lucene.codecs.lucene80}
+ * Components from the Lucene 7.0 index format.  See {@link org.apache.lucene.codecs.lucene86}
  * for an overview of the current index format.
  */
 package org.apache.lucene.codecs.lucene70;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
similarity index 96%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
copy to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
index e3f061a..579c6a0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
@@ -60,31 +60,31 @@ public class Lucene84Codec extends Codec {
   private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
   private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
   private final PostingsFormat defaultFormat;
-  
+
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
     @Override
     public PostingsFormat getPostingsFormatForField(String field) {
       return Lucene84Codec.this.getPostingsFormatForField(field);
     }
   };
-  
+
   private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
     @Override
     public DocValuesFormat getDocValuesFormatForField(String field) {
       return Lucene84Codec.this.getDocValuesFormatForField(field);
     }
   };
-  
+
   private final StoredFieldsFormat storedFieldsFormat;
 
-  /** 
+  /**
    * Instantiates a new codec.
    */
   public Lucene84Codec() {
     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
@@ -95,12 +95,12 @@ public class Lucene84Codec extends Codec {
     this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
     this.defaultFormat = new Lucene84PostingsFormat();
   }
-  
+
   @Override
   public final StoredFieldsFormat storedFieldsFormat() {
     return storedFieldsFormat;
   }
-  
+
   @Override
   public final TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
@@ -110,17 +110,17 @@ public class Lucene84Codec extends Codec {
   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;
@@ -136,32 +136,32 @@ public class Lucene84Codec extends Codec {
     return new Lucene60PointsFormat();
   }
 
-  /** Returns the postings format that should be used for writing 
+  /** Returns the postings format that should be used for writing
    *  new segments of <code>field</code>.
-   *  
+   *
    *  The default implementation always returns "Lucene84".
    *  <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. 
+   *  <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 
+
+  /** Returns the docvalues format that should be used for writing
    *  new segments of <code>field</code>.
-   *  
+   *
    *  The default implementation always returns "Lucene80".
    *  <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. 
+   *  <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;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene84/package-info.java
similarity index 86%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
rename to lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene84/package-info.java
index e1913a0..5940a47 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/package-info.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene84/package-info.java
@@ -16,7 +16,7 @@
  */
 
 /**
- * Components from the Lucene 7.0 index format.  See {@link org.apache.lucene.codecs.lucene80}
+ * Components from the Lucene 8.4 index format.  See {@link org.apache.lucene.codecs.lucene86}
  * for an overview of the current index format.
  */
-package org.apache.lucene.codecs.lucene70;
+package org.apache.lucene.codecs.lucene84;
diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index a818e35..cf7a945 100644
--- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene80.Lucene80Codec
+org.apache.lucene.codecs.lucene84.Lucene84Codec
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWSegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWSegmentInfoFormat.java
new file mode 100644
index 0000000..75f31c2
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWSegmentInfoFormat.java
@@ -0,0 +1,204 @@
+/*
+ * 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.Set;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortedNumericSelector;
+import org.apache.lucene.search.SortedNumericSortField;
+import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.search.SortedSetSortField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Version;
+
+/**
+ * Writable version of Lucene70SegmentInfoFormat for testing
+ */
+public class Lucene70RWSegmentInfoFormat extends Lucene70SegmentInfoFormat {
+
+  @Override
+  public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene70SegmentInfoFormat.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,
+          Lucene70SegmentInfoFormat.CODEC_NAME,
+          Lucene70SegmentInfoFormat.VERSION_CURRENT,
+          si.getId(),
+          "");
+      Version version = si.getVersion();
+      if (version.major < 7) {
+        throw new IllegalArgumentException("invalid major version: should be >= 7 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);
+
+      // Write the min Lucene version that contributed docs to the segment, since 7.0
+      if (si.getMinVersion() != null) {
+        output.writeByte((byte) 1);
+        Version minVersion = si.getMinVersion();
+        output.writeInt(minVersion.major);
+        output.writeInt(minVersion.minor);
+        output.writeInt(minVersion.bugfix);
+      } else {
+        output.writeByte((byte) 0);
+      }
+
+      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];
+        SortField.Type sortType = sortField.getType();
+        output.writeString(sortField.getField());
+        int sortTypeID;
+        switch (sortField.getType()) {
+          case STRING:
+            sortTypeID = 0;
+            break;
+          case LONG:
+            sortTypeID = 1;
+            break;
+          case INT:
+            sortTypeID = 2;
+            break;
+          case DOUBLE:
+            sortTypeID = 3;
+            break;
+          case FLOAT:
+            sortTypeID = 4;
+            break;
+          case CUSTOM:
+            if (sortField instanceof SortedSetSortField) {
+              sortTypeID = 5;
+              sortType = SortField.Type.STRING;
+            } else if (sortField instanceof SortedNumericSortField) {
+              sortTypeID = 6;
+              sortType = ((SortedNumericSortField) sortField).getNumericType();
+            } else {
+              throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField);
+            }
+            break;
+          default:
+            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
+        }
+        output.writeVInt(sortTypeID);
+        if (sortTypeID == 5) {
+          SortedSetSortField ssf = (SortedSetSortField) sortField;
+          if (ssf.getSelector() == SortedSetSelector.Type.MIN) {
+            output.writeByte((byte) 0);
+          } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) {
+            output.writeByte((byte) 1);
+          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) {
+            output.writeByte((byte) 2);
+          } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) {
+            output.writeByte((byte) 3);
+          } else {
+            throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector());
+          }
+        } else if (sortTypeID == 6) {
+          SortedNumericSortField snsf = (SortedNumericSortField) sortField;
+          if (snsf.getNumericType() == SortField.Type.LONG) {
+            output.writeByte((byte) 0);
+          } else if (snsf.getNumericType() == SortField.Type.INT) {
+            output.writeByte((byte) 1);
+          } else if (snsf.getNumericType() == SortField.Type.DOUBLE) {
+            output.writeByte((byte) 2);
+          } else if (snsf.getNumericType() == SortField.Type.FLOAT) {
+            output.writeByte((byte) 3);
+          } else {
+            throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType());
+          }
+          if (snsf.getSelector() == SortedNumericSelector.Type.MIN) {
+            output.writeByte((byte) 0);
+          } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) {
+            output.writeByte((byte) 1);
+          } else {
+            throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector());
+          }
+        }
+        output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
+
+        // write missing value
+        Object missingValue = sortField.getMissingValue();
+        if (missingValue == null) {
+          output.writeByte((byte) 0);
+        } else {
+          switch(sortType) {
+            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;
+            case DOUBLE:
+              output.writeByte((byte) 1);
+              output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue()));
+              break;
+            case FLOAT:
+              output.writeByte((byte) 1);
+              output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue()));
+              break;
+            default:
+              throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
+          }
+        }
+      }
+
+      CodecUtil.writeFooter(output);
+    }
+  }
+
+}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java
similarity index 77%
copy from lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java
copy to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java
index 3bf6a18..ac516a1 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java
@@ -14,22 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.lucene.codecs.lucene70;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
-import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 
 public class TestLucene70SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
 
   @Override
   protected Version[] getVersions() {
-    return new Version[] { Version.LATEST };
+    return new Version[] { Version.LUCENE_8_4_0 };
   }
 
   @Override
   protected Codec getCodec() {
-    return TestUtil.getDefaultCodec();
+    return new FilterCodec("Lucene84", Codec.forName("Lucene84")) {
+      @Override
+      public SegmentInfoFormat segmentInfoFormat() {
+        return new Lucene70RWSegmentInfoFormat();
+      }
+    };
   }
 }
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 5510328..db64781 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.lucene84.Lucene84Codec;
+import org.apache.lucene.codecs.lucene86.Lucene86Codec;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
@@ -138,7 +138,7 @@ public class CreateIndexTask extends PerfTask {
     if (defaultCodec == null && postingsFormat != null) {
       try {
         final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
-        iwConf.setCodec(new Lucene84Codec() {
+        iwConf.setCodec(new Lucene86Codec() {
           @Override
           public PostingsFormat getPostingsFormatForField(String field) {
             return postingsFormatChosen;
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
index 5f22f62..2acfe01 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
@@ -29,17 +29,16 @@ import java.util.Set;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexSorter;
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SortFieldProvider;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSelector;
-import org.apache.lucene.search.SortedNumericSortField;
-import org.apache.lucene.search.SortedSetSelector;
-import org.apache.lucene.search.SortedSetSortField;
+import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -68,11 +67,9 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
   final static BytesRef SI_FILE             = new BytesRef("      file ");
   final static BytesRef SI_ID               = new BytesRef("    id ");
   final static BytesRef SI_SORT             = new BytesRef("    sort ");
-  final static BytesRef SI_SORT_FIELD       = new BytesRef("      field ");
   final static BytesRef SI_SORT_TYPE        = new BytesRef("      type ");
-  final static BytesRef SI_SELECTOR_TYPE    = new BytesRef("      selector ");
-  final static BytesRef SI_SORT_REVERSE     = new BytesRef("      reverse ");
-  final static BytesRef SI_SORT_MISSING     = new BytesRef("      missing ");
+  final static BytesRef SI_SORT_NAME        = new BytesRef("      name ");
+  final static BytesRef SI_SORT_BYTES       = new BytesRef("      bytes ");
 
   public static final String SI_EXTENSION = "si";
   
@@ -171,133 +168,18 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
       SortField[] sortField = new SortField[numSortFields];
       for (int i = 0; i < numSortFields; ++i) {
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch.get(), SI_SORT_FIELD);
-        final String field = readString(SI_SORT_FIELD.length, scratch);
+        assert StringHelper.startsWith(scratch.get(), SI_SORT_NAME);
+        final String provider = readString(SI_SORT_NAME.length, scratch);
 
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch.get(), SI_SORT_TYPE);
-        final String typeAsString = readString(SI_SORT_TYPE.length, scratch);
-
-        final SortField.Type type;
-        SortedSetSelector.Type selectorSet = null;
-        SortedNumericSelector.Type selectorNumeric = null;
-        switch (typeAsString) {
-          case "string":
-            type = SortField.Type.STRING;
-            break;
-          case "long":
-            type = SortField.Type.LONG;
-            break;
-          case "int":
-            type = SortField.Type.INT;
-            break;
-          case "double":
-            type = SortField.Type.DOUBLE;
-            break;
-          case "float":
-            type = SortField.Type.FLOAT;
-            break;
-          case "multi_valued_string":
-            type = SortField.Type.STRING;
-            selectorSet = readSetSelector(input, scratch);
-            break;
-          case "multi_valued_long":
-            type = SortField.Type.LONG;
-            selectorNumeric = readNumericSelector(input, scratch);
-            break;
-          case "multi_valued_int":
-            type = SortField.Type.INT;
-            selectorNumeric = readNumericSelector(input, scratch);
-            break;
-          case "multi_valued_double":
-            type = SortField.Type.DOUBLE;
-            selectorNumeric = readNumericSelector(input, scratch);
-            break;
-          case "multi_valued_float":
-            type = SortField.Type.FLOAT;
-            selectorNumeric = readNumericSelector(input, scratch);
-            break;
-          default:
-            throw new CorruptIndexException("unable to parse sort type string: " + typeAsString, input);
-        }
 
         SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch.get(), SI_SORT_REVERSE);
-        final boolean reverse = Boolean.parseBoolean(readString(SI_SORT_REVERSE.length, scratch));
-
-        SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch.get(), SI_SORT_MISSING);
-        final String missingLastAsString = readString(SI_SORT_MISSING.length, scratch);
-        final Object missingValue;
-        switch (type) {
-          case STRING:
-            switch (missingLastAsString) {
-              case "null":
-                missingValue = null;
-                break;
-              case "first":
-                missingValue = SortField.STRING_FIRST;
-                break;
-              case "last":
-                missingValue = SortField.STRING_LAST;
-                break;
-              default:
-                throw new CorruptIndexException("unable to parse missing string: " + typeAsString, input);
-            }
-            break;
-          case LONG:
-            switch (missingLastAsString) {
-              case "null":
-                missingValue = null;
-                break;
-              default:
-                missingValue = Long.parseLong(missingLastAsString);
-                break;
-            }
-            break;
-          case INT:
-            switch (missingLastAsString) {
-              case "null":
-                missingValue = null;
-                break;
-              default:
-                missingValue = Integer.parseInt(missingLastAsString);
-                break;
-            }
-            break;
-          case DOUBLE:
-            switch (missingLastAsString) {
-              case "null":
-                missingValue = null;
-                break;
-              default:
-                missingValue = Double.parseDouble(missingLastAsString);
-                break;
-            }
-            break;
-          case FLOAT:
-            switch (missingLastAsString) {
-              case "null":
-                missingValue = null;
-                break;
-              default:
-                missingValue = Float.parseFloat(missingLastAsString);
-                break;
-            }
-            break;
-          default:
-            throw new AssertionError();
-        }
-        if (selectorSet != null) {
-          sortField[i] = new SortedSetSortField(field, reverse);
-        } else if (selectorNumeric != null) {
-          sortField[i] = new SortedNumericSortField(field, type, reverse);
-        } else {
-          sortField[i] = new SortField(field, type, reverse);
-        }
-        if (missingValue != null) {
-          sortField[i].setMissingValue(missingValue);
-        }
+        assert StringHelper.startsWith(scratch.get(), SI_SORT_BYTES);
+        BytesRef serializedSort = SimpleTextUtil.fromBytesRefString(readString(SI_SORT_BYTES.length, scratch));
+        final ByteArrayDataInput bytes = new ByteArrayDataInput(serializedSort.bytes, serializedSort.offset, serializedSort.length);
+        sortField[i] = SortFieldProvider.forName(provider).readSortField(bytes);
+        assert bytes.eof();
       }
       Sort indexSort = sortField.length == 0 ? null : new Sort(sortField);
 
@@ -313,38 +195,6 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
   private String readString(int offset, BytesRefBuilder scratch) {
     return new String(scratch.bytes(), offset, scratch.length()-offset, StandardCharsets.UTF_8);
   }
-
-  private SortedSetSelector.Type readSetSelector(IndexInput input, BytesRefBuilder scratch) throws IOException {
-    SimpleTextUtil.readLine(input, scratch);
-    assert StringHelper.startsWith(scratch.get(), SI_SELECTOR_TYPE);
-    final String selectorAsString = readString(SI_SELECTOR_TYPE.length, scratch);
-    switch (selectorAsString) {
-      case "min":
-        return SortedSetSelector.Type.MIN;
-      case "middle_min":
-        return SortedSetSelector.Type.MIDDLE_MIN;
-      case "middle_max":
-        return SortedSetSelector.Type.MIDDLE_MAX;
-      case "max":
-        return SortedSetSelector.Type.MAX;
-      default:
-        throw new CorruptIndexException("unable to parse SortedSetSelector type: " + selectorAsString, input);
-    }
-  }
-
-  private SortedNumericSelector.Type readNumericSelector(IndexInput input, BytesRefBuilder scratch) throws IOException {
-    SimpleTextUtil.readLine(input, scratch);
-    assert StringHelper.startsWith(scratch.get(), SI_SELECTOR_TYPE);
-    final String selectorAsString = readString(SI_SELECTOR_TYPE.length, scratch);
-    switch (selectorAsString) {
-      case "min":
-        return SortedNumericSelector.Type.MIN;
-      case "max":
-        return SortedNumericSelector.Type.MAX;
-      default:
-        throw new CorruptIndexException("unable to parse SortedNumericSelector type: " + selectorAsString, input);
-    }
-  }
   
   @Override
   public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
@@ -434,120 +284,42 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
       SimpleTextUtil.writeNewline(output);
       for (int i = 0; i < numSortFields; ++i) {
         final SortField sortField = indexSort.getSort()[i];
+        IndexSorter sorter = sortField.getIndexSorter();
+        if (sorter == null) {
+          throw new IllegalStateException("Cannot serialize sort " + sortField);
+        }
 
-        SimpleTextUtil.write(output, SI_SORT_FIELD);
-        SimpleTextUtil.write(output, sortField.getField(), scratch);
+        SimpleTextUtil.write(output, SI_SORT_NAME);
+        SimpleTextUtil.write(output, sorter.getProviderName(), scratch);
         SimpleTextUtil.writeNewline(output);
 
         SimpleTextUtil.write(output, SI_SORT_TYPE);
-        final String sortTypeString;
-        final SortField.Type sortType;
-        final boolean multiValued;
-        if (sortField instanceof SortedSetSortField) {
-          sortType = SortField.Type.STRING;
-          multiValued = true;
-        } else if (sortField instanceof SortedNumericSortField) {
-          sortType = ((SortedNumericSortField) sortField).getNumericType();
-          multiValued = true;
-        } else {
-          sortType = sortField.getType();
-          multiValued = false;
-        }
-        switch (sortType) {
-          case STRING:
-              if (multiValued) {
-                sortTypeString = "multi_valued_string";
-              } else {
-                sortTypeString = "string";
-              }
-            break;
-          case LONG:
-            if (multiValued) {
-              sortTypeString = "multi_valued_long";
-            } else {
-              sortTypeString = "long";
-            }
-            break;
-          case INT:
-            if (multiValued) {
-              sortTypeString = "multi_valued_int";
-            } else {
-              sortTypeString = "int";
-            }
-            break;
-          case DOUBLE:
-            if (multiValued) {
-              sortTypeString = "multi_valued_double";
-            } else {
-              sortTypeString = "double";
-            }
-            break;
-          case FLOAT:
-            if (multiValued) {
-              sortTypeString = "multi_valued_float";
-            } else {
-              sortTypeString = "float";
-            }
-            break;
-          default:
-            throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
-        }
-        SimpleTextUtil.write(output, sortTypeString, scratch);
-        SimpleTextUtil.writeNewline(output);
-
-        if (sortField instanceof SortedSetSortField) {
-          SortedSetSelector.Type selector = ((SortedSetSortField) sortField).getSelector();
-          final String selectorString;
-          if (selector == SortedSetSelector.Type.MIN) {
-            selectorString = "min";
-          } else if (selector == SortedSetSelector.Type.MIDDLE_MIN) {
-            selectorString = "middle_min";
-          } else if (selector == SortedSetSelector.Type.MIDDLE_MAX) {
-            selectorString = "middle_max";
-          } else if (selector == SortedSetSelector.Type.MAX) {
-            selectorString = "max";
-          } else {
-            throw new IllegalStateException("Unexpected SortedSetSelector type selector: " + selector);
-          }
-          SimpleTextUtil.write(output, SI_SELECTOR_TYPE);
-          SimpleTextUtil.write(output, selectorString, scratch);
-          SimpleTextUtil.writeNewline(output);
-        } else if (sortField instanceof SortedNumericSortField) {
-          SortedNumericSelector.Type selector = ((SortedNumericSortField) sortField).getSelector();
-          final String selectorString;
-          if (selector == SortedNumericSelector.Type.MIN) {
-            selectorString = "min";
-          } else if (selector == SortedNumericSelector.Type.MAX) {
-            selectorString = "max";
-          } else {
-            throw new IllegalStateException("Unexpected SortedNumericSelector type selector: " + selector);
-          }
-          SimpleTextUtil.write(output, SI_SELECTOR_TYPE);
-          SimpleTextUtil.write(output, selectorString, scratch);
-          SimpleTextUtil.writeNewline(output);
-        }
-
-        SimpleTextUtil.write(output, SI_SORT_REVERSE);
-        SimpleTextUtil.write(output, Boolean.toString(sortField.getReverse()), scratch);
+        SimpleTextUtil.write(output, sortField.toString(), scratch);
         SimpleTextUtil.writeNewline(output);
 
-        SimpleTextUtil.write(output, SI_SORT_MISSING);
-        final Object missingValue = sortField.getMissingValue();
-        final String missing;
-        if (missingValue == null) {
-          missing = "null";
-        } else if (missingValue == SortField.STRING_FIRST) {
-          missing = "first";
-        } else if (missingValue == SortField.STRING_LAST) {
-          missing = "last";
-        } else {
-          missing = missingValue.toString();
-        }
-        SimpleTextUtil.write(output, missing, scratch);
+        SimpleTextUtil.write(output, SI_SORT_BYTES);
+        BytesRefOutput b = new BytesRefOutput();
+        SortFieldProvider.write(sortField, b);
+        SimpleTextUtil.write(output, b.bytes.get().toString(), scratch);
         SimpleTextUtil.writeNewline(output);
       }
       
       SimpleTextUtil.writeChecksum(output, scratch);
     }
   }
+
+  static class BytesRefOutput extends DataOutput {
+
+    final BytesRefBuilder bytes = new BytesRefBuilder();
+
+    @Override
+    public void writeByte(byte b) {
+      bytes.append(b);
+    }
+
+    @Override
+    public void writeBytes(byte[] b, int offset, int length) {
+      bytes.append(b, offset, length);
+    }
+  }
 }
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 07797c6..8b5ca14 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("Lucene84");
+    static Codec defaultCodec = LOADER.lookup("Lucene86");
   }
 
   private final String name;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java
index 91ee2e2..5940a47 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java
@@ -16,399 +16,7 @@
  */
 
 /**
- * Lucene 8.4 file format.
- * 
- * <h2>Apache Lucene - Index File Formats</h2>
- * <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 id="Introduction"></a>
- * <h3>Introduction</h3>
- * <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 id="Definitions"></a>
- * <h3>Definitions</h3>
- * <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 id="Inverted_Indexing"></a>
- * <h4>Inverted Indexing</h4>
- * <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 id="Types_of_Fields"></a>
- * <h4>Types of Fields</h4>
- * <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 id="Segments"></a>
- * <h4>Segments</h4>
- * <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 id="Document_Numbers"></a>
- * <h4>Document Numbers</h4>
- * <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 id="Overview"></a>
- * <h3>Index Structure Overview</h3>
- * <div>
- * <p>Each segment index maintains the following:</p>
- * <ul>
- * <li>
- * {@link org.apache.lucene.codecs.lucene70.Lucene70SegmentInfoFormat 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.lucene60.Lucene60FieldInfosFormat 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.lucene84.Lucene84PostingsFormat 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.lucene84.Lucene84PostingsFormat 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 ({@link org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
- * </li>
- * <li>
- * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat 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.lucene80.Lucene80NormsFormat 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.lucene80.Lucene80DocValuesFormat 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>
- * <li>
- * {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
- * Optional pair of files, recording dimensionally indexed fields, to enable fast
- * numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
- * and geographic shape intersection (2D, 3D).
- * </li>
- * </ul>
- * <p>Details on each of these are provided in their linked pages.</p>
- * </div>
- * <a id="File_Naming"></a>
- * <h3>File Naming</h3>
- * <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 id="file-names"></a>
- * <h3>Summary of File Extensions</h3>
- * <div>
- * <p>The following table summarizes the names and extensions of the files in
- * Lucene:</p>
- * <table class="padding4" style="border-spacing: 1px; border-collapse: separate">
- * <caption>lucene filenames by extension</caption>
- * <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.lucene70.Lucene70SegmentInfoFormat 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.lucene60.Lucene60FieldInfosFormat 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.lucene84.Lucene84PostingsFormat Term Dictionary}</td>
- * <td>.tim</td>
- * <td>The term dictionary, stores term info</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Index}</td>
- * <td>.tip</td>
- * <td>The index into the Term Dictionary</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat 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.lucene84.Lucene84PostingsFormat 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.lucene84.Lucene84PostingsFormat 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.lucene80.Lucene80NormsFormat 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.lucene80.Lucene80DocValuesFormat 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 documents are live</td>
- * </tr>
- * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
- * <td>.dii, .dim</td>
- * <td>Holds indexed points, if any</td>
- * </tr>
- * </table>
- * </div>
- * <a id="Lock_File"></a>
- * <h3>Lock File</h3>
- * 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 id="History"></a>
- * <h3>History</h3>
- * <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 6.0, Points were added, for multi-dimensional range/distance search.
- * <li>In version 6.2, new Segment info format that reads/writes the index sort, to support index sorting.
- * <li>In version 7.0, DocValues have been improved to better support sparse doc values
- * thanks to an iterator API.</li>
- * <li>In version 8.0, postings have been enhanced to record, for each block of
- * doc ids, the (term freq, normalization factor) pairs that may trigger the
- * maximum score of the block. This information is recorded alongside skip data
- * in order to be able to skip blocks of doc ids if they may not produce high
- * enough scores.
- * Additionally doc values and norms has been extended with jump-tables to make access O(1)
- * instead of O(n), where n is the number of elements to skip when advancing in the data.</li>
- * <li>In version 8.4, postings, positions, offsets and payload lengths have move to a more
- * performant encoding that is vectorized.</li>
- * </ul>
- * <a id="Limitations"></a>
- * <h3>Limitations</h3>
- * <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>
+ * Components from the Lucene 8.4 index format.  See {@link org.apache.lucene.codecs.lucene86}
+ * for an overview of the current index format.
  */
 package org.apache.lucene.codecs.lucene84;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86Codec.java
similarity index 86%
rename from lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86Codec.java
index e3f061a..b9116e4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86Codec.java
@@ -14,7 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.codecs.lucene84;
+
+package org.apache.lucene.codecs.lucene86;
 
 import java.util.Objects;
 
@@ -33,74 +34,73 @@ 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.lucene60.Lucene60FieldInfosFormat;
 import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
-import org.apache.lucene.codecs.lucene70.Lucene70SegmentInfoFormat;
 import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
+import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
 /**
- * Implements the Lucene 8.4 index format, with configurable per-field postings
+ * Implements the Lucene 8.6 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.lucene84 package documentation for file format details.
+ * @see org.apache.lucene.codecs.lucene86 package documentation for file format details.
  *
  * @lucene.experimental
  */
-public class Lucene84Codec extends Codec {
+public class Lucene86Codec extends Codec {
   private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
-  private final SegmentInfoFormat segmentInfosFormat = new Lucene70SegmentInfoFormat();
+  private final SegmentInfoFormat segmentInfosFormat = new Lucene86SegmentInfoFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
   private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
   private final PostingsFormat defaultFormat;
-  
+
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
     @Override
     public PostingsFormat getPostingsFormatForField(String field) {
-      return Lucene84Codec.this.getPostingsFormatForField(field);
+      return Lucene86Codec.this.getPostingsFormatForField(field);
     }
   };
-  
+
   private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
     @Override
     public DocValuesFormat getDocValuesFormatForField(String field) {
-      return Lucene84Codec.this.getDocValuesFormatForField(field);
+      return Lucene86Codec.this.getDocValuesFormatForField(field);
     }
   };
-  
+
   private final StoredFieldsFormat storedFieldsFormat;
 
-  /** 
+  /**
    * Instantiates a new codec.
    */
-  public Lucene84Codec() {
-    this(Mode.BEST_SPEED);
+  public Lucene86Codec() {
+    this(Lucene50StoredFieldsFormat.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 Lucene84Codec(Mode mode) {
-    super("Lucene84");
+  public Lucene86Codec(Lucene50StoredFieldsFormat.Mode mode) {
+    super("Lucene86");
     this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
     this.defaultFormat = new Lucene84PostingsFormat();
   }
-  
+
   @Override
   public final StoredFieldsFormat storedFieldsFormat() {
     return storedFieldsFormat;
   }
-  
+
   @Override
   public final TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
@@ -110,17 +110,17 @@ public class Lucene84Codec extends Codec {
   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;
@@ -136,32 +136,32 @@ public class Lucene84Codec extends Codec {
     return new Lucene60PointsFormat();
   }
 
-  /** Returns the postings format that should be used for writing 
+  /** Returns the postings format that should be used for writing
    *  new segments of <code>field</code>.
-   *  
+   *
    *  The default implementation always returns "Lucene84".
    *  <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. 
+   *  <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 
+
+  /** Returns the docvalues format that should be used for writing
    *  new segments of <code>field</code>.
-   *  
+   *
    *  The default implementation always returns "Lucene80".
    *  <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. 
+   *  <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;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86SegmentInfoFormat.java
new file mode 100644
index 0000000..b2bcdc2
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/Lucene86SegmentInfoFormat.java
@@ -0,0 +1,217 @@
+/*
+ * 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.lucene86;
+
+import java.io.IOException;
+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.IndexSorter;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.SortFieldProvider;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Version;
+
+/**
+ * Lucene 8.6 Segment info format.
+ * <p>
+ * Files:
+ * <ul>
+ *   <li><code>.si</code>: 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>SegMinVersion --&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#writeVInt Int32} count, followed by {@code count} SortField</li>
+ *   <li>SortField --&gt; {@link DataOutput#writeString String} sort class, followed by a per-sort bytestream
+ *    (see {@link SortFieldProvider#readSortField(DataInput)})
+ *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ * </ul>
+ * Field Descriptions:
+ * <ul>
+ *   <li>SegVersion is the code version that created the segment.</li>
+ *   <li>SegMinVersion is the minimum code version that contributed documents to 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 Lucene86SegmentInfoFormat extends SegmentInfoFormat {
+
+  /** Sole constructor. */
+  public Lucene86SegmentInfoFormat() {
+  }
+
+  @Override
+  public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segment, "", SI_EXTENSION);
+    try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) {
+      Throwable priorE = null;
+      SegmentInfo si = null;
+      try {
+        int format = CodecUtil.checkIndexHeader(input, CODEC_NAME,
+            VERSION_START,
+            VERSION_CURRENT,
+            segmentID, "");
+        final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+        byte hasMinVersion = input.readByte();
+        final Version minVersion;
+        switch (hasMinVersion) {
+          case 0:
+            minVersion = null;
+            break;
+          case 1:
+            minVersion = Version.fromBits(input.readInt(), input.readInt(), input.readInt());
+            break;
+          default:
+            throw new CorruptIndexException("Illegal boolean value " + hasMinVersion, input);
+        }
+
+        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 name = input.readString();
+            sortFields[i] = SortFieldProvider.forName(name).readSortField(input);
+          }
+          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, minVersion, 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, "", 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,
+          CODEC_NAME,
+          VERSION_CURRENT,
+          si.getId(),
+          "");
+      Version version = si.getVersion();
+      if (version.major < 7) {
+        throw new IllegalArgumentException("invalid major version: should be >= 7 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);
+
+      // Write the min Lucene version that contributed docs to the segment, since 7.0
+      if (si.getMinVersion() != null) {
+        output.writeByte((byte) 1);
+        Version minVersion = si.getMinVersion();
+        output.writeInt(minVersion.major);
+        output.writeInt(minVersion.minor);
+        output.writeInt(minVersion.bugfix);
+      } else {
+        output.writeByte((byte) 0);
+      }
+
+      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];
+        IndexSorter sorter = sortField.getIndexSorter();
+        if (sorter == null) {
+          throw new IllegalArgumentException("cannot serialize SortField " + sortField);
+        }
+        output.writeString(sorter.getProviderName());
+        SortFieldProvider.write(sortField, output);
+      }
+
+      CodecUtil.writeFooter(output);
+    }
+  }
+
+  /** File extension used to store {@link SegmentInfo}. */
+  public final static String SI_EXTENSION = "si";
+  static final String CODEC_NAME = "Lucene86SegmentInfo";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/package-info.java
similarity index 95%
copy from lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java
copy to lucene/core/src/java/org/apache/lucene/codecs/lucene86/package-info.java
index 91ee2e2..ed6b96c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene86/package-info.java
@@ -16,8 +16,8 @@
  */
 
 /**
- * Lucene 8.4 file format.
- * 
+ * Lucene 8.6 file format.
+ *
  * <h2>Apache Lucene - Index File Formats</h2>
  * <div>
  * <ul>
@@ -61,7 +61,7 @@
  * <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 
+ * <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 id="Inverted_Indexing"></a>
@@ -128,36 +128,36 @@
  * <p>Each segment index maintains the following:</p>
  * <ul>
  * <li>
- * {@link org.apache.lucene.codecs.lucene70.Lucene70SegmentInfoFormat Segment info}.
+ * {@link org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat Segment info}.
  *    This contains metadata about a segment, such as the number of documents,
- *    what files it uses, 
+ *    what files it uses, and information about how the segment is sorted
  * </li>
  * <li>
- * {@link org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat Field names}. 
+ * {@link org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat 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 
+ * {@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.lucene84.Lucene84PostingsFormat Term dictionary}. 
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat 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.lucene84.Lucene84PostingsFormat Term Frequency data}. 
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat 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 ({@link org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS})
  * </li>
  * <li>
- * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat Term Proximity data}. 
+ * {@link org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat 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.
@@ -168,10 +168,10 @@
  * that is multiplied into the score for hits on that field.
  * </li>
  * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}. 
+ * {@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 
+ * term frequency. To add Term Vectors to your index see the
  * {@link org.apache.lucene.document.Field Field} constructors
  * </li>
  * <li>
@@ -182,7 +182,7 @@
  * searches, per-document values are useful for things like scoring factors.
  * </li>
  * <li>
- * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}. 
+ * {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
  * An optional file indicating which documents are live.
  * </li>
  * <li>
@@ -200,7 +200,7 @@
  * <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 
+ * 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>
@@ -234,7 +234,7 @@
  * file.</td>
  * </tr>
  * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene70.Lucene70SegmentInfoFormat Segment Info}</td>
+ * <td>{@link org.apache.lucene.codecs.lucene86.Lucene86SegmentInfoFormat Segment Info}</td>
  * <td>.si</td>
  * <td>Stores metadata about a segment</td>
  * </tr>
@@ -351,12 +351,12 @@
  * <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> 
+ * 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> 
+ * <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>
@@ -364,22 +364,22 @@
  * 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 
+ * ({@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 
+ * 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 
+ * <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 
+ * <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 
+ * <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)
@@ -399,6 +399,8 @@
  * instead of O(n), where n is the number of elements to skip when advancing in the data.</li>
  * <li>In version 8.4, postings, positions, offsets and payload lengths have move to a more
  * performant encoding that is vectorized.</li>
+ * <li>In version 8.6, index sort serialization is delegated to the sorts themselves, to
+ * allow user-defined sorts to be used</li>
  * </ul>
  * <a id="Limitations"></a>
  * <h3>Limitations</h3>
@@ -411,4 +413,4 @@
  * better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
  * </div>
  */
-package org.apache.lucene.codecs.lucene84;
+package org.apache.lucene.codecs.lucene86;
diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
index 1aeab4c..e213a48 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
@@ -37,7 +36,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 /** Buffers up pending byte[] per doc, then flushes when
  *  segment flushes. */
-class BinaryDocValuesWriter extends DocValuesWriter {
+class BinaryDocValuesWriter extends DocValuesWriter<BinaryDocValues> {
 
   /** Maximum length for a binary field. */
   private static final int MAX_LENGTH = ArrayUtil.MAX_ARRAY_LENGTH;
@@ -56,6 +55,8 @@ class BinaryDocValuesWriter extends DocValuesWriter {
   private int lastDocID = -1;
   private int maxLength = 0;
 
+  private PackedLongValues finalLengths;
+
   public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     this.fieldInfo = fieldInfo;
     this.bytes = new PagedBytes(BLOCK_BITS);
@@ -98,10 +99,6 @@ class BinaryDocValuesWriter extends DocValuesWriter {
     bytesUsed = newBytesUsed;
   }
 
-  @Override
-  public void finish(int maxDoc) {
-  }
-
   private SortingLeafReader.CachedBinaryDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, BinaryDocValues oldValues) throws IOException {
     FixedBitSet docsWithField = new FixedBitSet(maxDoc);
     BytesRef[] values = new BytesRef[maxDoc];
@@ -118,18 +115,23 @@ class BinaryDocValuesWriter extends DocValuesWriter {
   }
 
   @Override
-  Sorter.DocComparator getDocComparator(int numDoc, SortField sortField) throws IOException {
-    throw new IllegalArgumentException("It is forbidden to sort on a binary field");
+  BinaryDocValues getDocValues() {
+    if (finalLengths == null) {
+      finalLengths = this.lengths.build();
+    }
+    return new BufferedBinaryDocValues(finalLengths, maxLength, bytes.getDataInput(), docsWithField.iterator());
   }
 
   @Override
   public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
     bytes.freeze(false);
-    final PackedLongValues lengths = this.lengths.build();
+    if (finalLengths == null) {
+      finalLengths = this.lengths.build();
+    }
     final SortingLeafReader.CachedBinaryDVs sorted;
     if (sortMap != null) {
       sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap,
-          new BufferedBinaryDocValues(lengths, maxLength, bytes.getDataInput(), docsWithField.iterator()));
+          new BufferedBinaryDocValues(finalLengths, maxLength, bytes.getDataInput(), docsWithField.iterator()));
     } else {
       sorted = null;
     }
@@ -141,7 +143,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
                                     throw new IllegalArgumentException("wrong fieldInfo");
                                   }
                                   if (sorted == null) {
-                                    return new BufferedBinaryDocValues(lengths, maxLength, bytes.getDataInput(), docsWithField.iterator());
+                                    return new BufferedBinaryDocValues(finalLengths, maxLength, bytes.getDataInput(), docsWithField.iterator());
                                   } else {
                                     return new SortingLeafReader.SortingBinaryDocValues(sorted);
                                   }
@@ -200,9 +202,4 @@ class BinaryDocValuesWriter extends DocValuesWriter {
       return value.get();
     }
   }
-
-  @Override
-  DocIdSetIterator getDocIdSet() {
-    return docsWithField.iterator();
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
index 03cabc1..cdd6ebe 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -22,10 +22,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
+import java.util.Objects;
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.codecs.DocValuesConsumer;
@@ -39,8 +38,6 @@ import org.apache.lucene.document.FieldType;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSortField;
-import org.apache.lucene.search.SortedSetSortField;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
@@ -75,8 +72,6 @@ final class DefaultIndexingChain extends DocConsumer {
   // Holds fields seen in each document
   private PerField[] fields = new PerField[1];
 
-  private final Set<String> finishedDocValues = new HashSet<>();
-
   public DefaultIndexingChain(DocumentsWriterPerThread docWriter) {
     this.docWriter = docWriter;
     this.fieldInfos = docWriter.getFieldInfosBuilder();
@@ -94,29 +89,96 @@ final class DefaultIndexingChain extends DocConsumer {
     termsHash = new FreqProxTermsWriter(docWriter, termVectorsWriter);
   }
 
+  private LeafReader getDocValuesLeafReader() {
+    return new DocValuesLeafReader() {
+      @Override
+      public NumericDocValues getNumericDocValues(String field) throws IOException {
+        PerField pf = getPerField(field);
+        if (pf == null) {
+          return null;
+        }
+        if (pf.fieldInfo.getDocValuesType() == DocValuesType.NUMERIC) {
+          return (NumericDocValues) pf.docValuesWriter.getDocValues();
+        }
+        return null;
+      }
+
+      @Override
+      public BinaryDocValues getBinaryDocValues(String field) throws IOException {
+        PerField pf = getPerField(field);
+        if (pf == null) {
+          return null;
+        }
+        if (pf.fieldInfo.getDocValuesType() == DocValuesType.BINARY) {
+          return (BinaryDocValues) pf.docValuesWriter.getDocValues();
+        }
+        return null;
+      }
+
+      @Override
+      public SortedDocValues getSortedDocValues(String field) throws IOException {
+        PerField pf = getPerField(field);
+        if (pf == null) {
+          return null;
+        }
+        if (pf.fieldInfo.getDocValuesType() == DocValuesType.SORTED) {
+          return (SortedDocValues) pf.docValuesWriter.getDocValues();
+        }
+        return null;
+      }
+
+      @Override
+      public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
+        PerField pf = getPerField(field);
+        if (pf == null) {
+          return null;
+        }
+        if (pf.fieldInfo.getDocValuesType() == DocValuesType.SORTED_NUMERIC) {
+          return (SortedNumericDocValues) pf.docValuesWriter.getDocValues();
+        }
+        return null;
+      }
+
+      @Override
+      public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
+        PerField pf = getPerField(field);
+        if (pf == null) {
+          return null;
+        }
+        if (pf.fieldInfo.getDocValuesType() == DocValuesType.SORTED_SET) {
+          return (SortedSetDocValues) pf.docValuesWriter.getDocValues();
+        }
+        return null;
+      }
+
+      @Override
+      public FieldInfos getFieldInfos() {
+        return fieldInfos.finish();
+      }
+
+    };
+  }
+
   private Sorter.DocMap maybeSortSegment(SegmentWriteState state) throws IOException {
     Sort indexSort = state.segmentInfo.getIndexSort();
     if (indexSort == null) {
       return null;
     }
 
-    List<Sorter.DocComparator> comparators = new ArrayList<>();
+    LeafReader docValuesReader = getDocValuesLeafReader();
+
+    List<IndexSorter.DocComparator> comparators = new ArrayList<>();
     for (int i = 0; i < indexSort.getSort().length; i++) {
       SortField sortField = indexSort.getSort()[i];
-      PerField perField = getPerField(sortField.getField());
-      if (perField != null && perField.docValuesWriter != null &&
-          finishedDocValues.contains(perField.fieldInfo.name) == false) {
-          perField.docValuesWriter.finish(state.segmentInfo.maxDoc());
-          Sorter.DocComparator cmp = perField.docValuesWriter.getDocComparator(state.segmentInfo.maxDoc(), sortField);
-          comparators.add(cmp);
-          finishedDocValues.add(perField.fieldInfo.name);
-      } else {
-        // safe to ignore, sort field with no values or already seen before
+      IndexSorter sorter = sortField.getIndexSorter();
+      if (sorter == null) {
+        throw new UnsupportedOperationException("Cannot sort index using sort field " + sortField);
       }
+      comparators.add(sorter.getDocComparator(docValuesReader, state.segmentInfo.maxDoc()));
     }
     Sorter sorter = new Sorter(indexSort);
     // returns null if the documents are already sorted
-    return sorter.sort(state.segmentInfo.maxDoc(), comparators.toArray(new Sorter.DocComparator[comparators.size()]));
+    return sorter.sort(state.segmentInfo.maxDoc(), comparators.toArray(IndexSorter.DocComparator[]::new));
   }
 
   @Override
@@ -255,10 +317,6 @@ final class DefaultIndexingChain extends DocConsumer {
               DocValuesFormat fmt = state.segmentInfo.getCodec().docValuesFormat();
               dvConsumer = fmt.fieldsConsumer(state);
             }
-
-            if (finishedDocValues.contains(perField.fieldInfo.name) == false) {
-              perField.docValuesWriter.finish(maxDoc);
-            }
             perField.docValuesWriter.flush(state, sortMap, dvConsumer);
             perField.docValuesWriter = null;
           } else if (perField.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
@@ -527,45 +585,58 @@ final class DefaultIndexingChain extends DocConsumer {
     fp.pointValuesWriter.addPackedValue(docState.docID, field.binaryValue());
   }
 
-  private void validateIndexSortDVType(Sort indexSort, String fieldName, DocValuesType dvType) {
+  private void validateIndexSortDVType(Sort indexSort, String fieldToValidate, DocValuesType dvType) throws IOException {
     for (SortField sortField : indexSort.getSort()) {
-      if (sortField.getField().equals(fieldName)) {
-        switch (dvType) {
-          case NUMERIC:
-            if (sortField.getType().equals(SortField.Type.INT) == false &&
-                  sortField.getType().equals(SortField.Type.LONG) == false &&
-                  sortField.getType().equals(SortField.Type.FLOAT) == false &&
-                  sortField.getType().equals(SortField.Type.DOUBLE) == false) {
-              throw new IllegalArgumentException("invalid doc value type:" + dvType + " for sortField:" + sortField);
-            }
-            break;
+      IndexSorter sorter = sortField.getIndexSorter();
+      if (sorter == null) {
+        throw new IllegalStateException("Cannot sort index with sort order " + sortField);
+      }
+      sorter.getDocComparator(new DocValuesLeafReader() {
+        @Override
+        public NumericDocValues getNumericDocValues(String field) {
+          if (Objects.equals(field, fieldToValidate) && dvType != DocValuesType.NUMERIC) {
+            throw new IllegalArgumentException("SortField " + sortField + " expected field [" + field + "] to be NUMERIC but it is [" + dvType + "]");
+          }
+          return DocValues.emptyNumeric();
+        }
 
-          case BINARY:
-            throw new IllegalArgumentException("invalid doc value type:" + dvType + " for sortField:" + sortField);
+        @Override
+        public BinaryDocValues getBinaryDocValues(String field) {
+          if (Objects.equals(field, fieldToValidate) && dvType != DocValuesType.BINARY) {
+            throw new IllegalArgumentException("SortField " + sortField + " expected field [" + field + "] to be BINARY but it is [" + dvType + "]");
+          }
+          return DocValues.emptyBinary();
+        }
 
-          case SORTED:
-            if (sortField.getType().equals(SortField.Type.STRING) == false) {
-              throw new IllegalArgumentException("invalid doc value type:" + dvType + " for sortField:" + sortField);
-            }
-            break;
+        @Override
+        public SortedDocValues getSortedDocValues(String field) {
+          if (Objects.equals(field, fieldToValidate) && dvType != DocValuesType.SORTED) {
+            throw new IllegalArgumentException("SortField " + sortField + " expected field [" + field + "] to be SORTED but it is [" + dvType + "]");
+          }
+          return DocValues.emptySorted();
+        }
 
-          case SORTED_NUMERIC:
-            if (sortField instanceof SortedNumericSortField == false) {
-              throw new IllegalArgumentException("invalid doc value type:" + dvType + " for sortField:" + sortField);
-            }
-            break;
+        @Override
+        public SortedNumericDocValues getSortedNumericDocValues(String field) {
+          if (Objects.equals(field, fieldToValidate) && dvType != DocValuesType.SORTED_NUMERIC) {
+            throw new IllegalArgumentException("SortField " + sortField + " expected field [" + field + "] to be SORTED_NUMERIC but it is [" + dvType + "]");
+          }
+          return DocValues.emptySortedNumeric();
+        }
 
-          case SORTED_SET:
-            if (sortField instanceof SortedSetSortField == false) {
-              throw new IllegalArgumentException("invalid doc value type:" + dvType + " for sortField:" + sortField);
-            }
-            break;
+        @Override
+        public SortedSetDocValues getSortedSetDocValues(String field) {
+          if (Objects.equals(field, fieldToValidate) && dvType != DocValuesType.SORTED_SET) {
+            throw new IllegalArgumentException("SortField " + sortField + " expected field [" + field + "] to be SORTED_SET but it is [" + dvType + "]");
+          }
+          return DocValues.emptySortedSet();
+        }
 
-          default:
-            throw new IllegalArgumentException("invalid doc value type:" + dvType + " for sortField:" + sortField);
+        @Override
+        public FieldInfos getFieldInfos() {
+          throw new UnsupportedOperationException();
         }
-        break;
-      }
+      }, 0);
     }
   }
 
@@ -581,8 +652,8 @@ final class DefaultIndexingChain extends DocConsumer {
         validateIndexSortDVType(indexSort, fp.fieldInfo.name, dvType);
       }
       fieldInfos.globalFieldNumbers.setDocValuesType(fp.fieldInfo.number, fp.fieldInfo.name, dvType);
-
     }
+
     fp.fieldInfo.setDocValuesType(dvType);
 
     int docID = docState.docID;
@@ -713,7 +784,7 @@ final class DefaultIndexingChain extends DocConsumer {
 
     // Non-null if this field ever had doc values in this
     // segment:
-    DocValuesWriter docValuesWriter;
+    DocValuesWriter<?> docValuesWriter;
 
     // Non-null if this field ever had points in this segment:
     PointValuesWriter pointValuesWriter;
@@ -907,7 +978,7 @@ final class DefaultIndexingChain extends DocConsumer {
           return null;
         }
 
-        return perField.docValuesWriter.getDocIdSet();
+        return perField.docValuesWriter.getDocValues();
       }
     }
     return null;
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesLeafReader.java
new file mode 100644
index 0000000..93b7f49
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesLeafReader.java
@@ -0,0 +1,89 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.util.Bits;
+
+abstract class DocValuesLeafReader extends LeafReader {
+  @Override
+  public final CacheHelper getCoreCacheHelper() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final Terms terms(String field) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final NumericDocValues getNormValues(String field) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final Bits getLiveDocs() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final PointValues getPointValues(String field) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final void checkIntegrity() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final LeafMetaData getMetaData() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final Fields getTermVectors(int docID) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final int numDocs() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final int maxDoc() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final void document(int docID, StoredFieldVisitor visitor) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected final void doClose() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final CacheHelper getReaderCacheHelper() {
+    throw new UnsupportedOperationException();
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesWriter.java
index b739b14..4098cb0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesWriter.java
@@ -21,12 +21,8 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.SortField;
 
-abstract class DocValuesWriter {
-  abstract void finish(int numDoc);
+abstract class DocValuesWriter<T extends DocIdSetIterator> {
   abstract void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer consumer) throws IOException;
-  abstract Sorter.DocComparator getDocComparator(int numDoc, SortField sortField) throws IOException;
-  abstract DocIdSetIterator getDocIdSet();
-
+  abstract T getDocValues();
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexSorter.java b/lucene/core/src/java/org/apache/lucene/index/IndexSorter.java
new file mode 100644
index 0000000..81fdf62
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexSorter.java
@@ -0,0 +1,448 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * Handles how documents should be sorted in an index, both within a segment and between
+ * segments.
+ *
+ * Implementers must provide the following methods:
+ * {@link #getDocComparator(LeafReader,int)} - an object that determines how documents within a segment are to be sorted
+ * {@link #getComparableProviders(List)} - an array of objects that return a sortable long value per document and segment
+ * {@link #getProviderName()} - the SPI-registered name of a {@link SortFieldProvider} to serialize the sort
+ *
+ * The companion {@link SortFieldProvider} should be registered with SPI via {@code META-INF/services}
+ */
+public interface IndexSorter {
+
+  /** Used for sorting documents across segments */
+  interface ComparableProvider {
+    /**
+     * Returns a long so that the natural ordering of long values matches the
+     * ordering of doc IDs for the given comparator
+     */
+    long getAsComparableLong(int docID) throws IOException;
+  }
+
+  /** A comparator of doc IDs, used for sorting documents within a segment */
+  interface DocComparator {
+    /** Compare docID1 against docID2. The contract for the return value is the
+     *  same as {@link Comparator#compare(Object, Object)}. */
+    int compare(int docID1, int docID2);
+  }
+
+  /**
+   * Get an array of {@link ComparableProvider}, one per segment, for merge sorting documents in different segments
+   * @param readers the readers to be merged
+   */
+  ComparableProvider[] getComparableProviders(List<? extends LeafReader> readers) throws IOException;
+
+  /**
+   * Get a comparator that determines the sort order of docs within a single Reader.
+   *
+   * NB We cannot simply use the {@link FieldComparator} API because it requires docIDs to be sent
+   * in-order. The default implementations allocate array[maxDoc] to hold native values for comparison,
+   * but 1) they are transient (only alive while sorting this one segment) and 2) in the typical
+   * index sorting case, they are only used to sort newly flushed segments, which will be smaller
+   * than merged segments
+   *
+   * @param reader the Reader to sort
+   * @param maxDoc the number of documents in the Reader
+   */
+  DocComparator getDocComparator(LeafReader reader, int maxDoc) throws IOException;
+
+  /**
+   * The SPI-registered name of a {@link SortFieldProvider} that will deserialize the parent SortField
+   */
+  String getProviderName();
+
+  /**
+   * Provide a NumericDocValues instance for a LeafReader
+   */
+  interface NumericDocValuesProvider {
+    /**
+     * Returns the NumericDocValues instance for this LeafReader
+     */
+    NumericDocValues get(LeafReader reader) throws IOException;
+  }
+
+  /**
+   * Provide a SortedDocValues instance for a LeafReader
+   */
+  interface SortedDocValuesProvider {
+    /**
+     * Returns the SortedDocValues instance for this LeafReader
+     */
+    SortedDocValues get(LeafReader reader) throws IOException;
+  }
+
+  /**
+   * Sorts documents based on integer values from a NumericDocValues instance
+   */
+  final class IntSorter implements IndexSorter {
+
+    private final Integer missingValue;
+    private final int reverseMul;
+    private final NumericDocValuesProvider valuesProvider;
+    private final String providerName;
+
+    /**
+     * Creates a new IntSorter
+     */
+    public IntSorter(String providerName, Integer missingValue, boolean reverse, NumericDocValuesProvider valuesProvider) {
+      this.missingValue = missingValue;
+      this.reverseMul = reverse ? -1 : 1;
+      this.valuesProvider = valuesProvider;
+      this.providerName = providerName;
+    }
+
+    @Override
+    public ComparableProvider[] getComparableProviders(List<? extends LeafReader> readers) throws IOException {
+      ComparableProvider[] providers = new ComparableProvider[readers.size()];
+      final long missingValue;
+      if (this.missingValue != null) {
+        missingValue = this.missingValue;
+      } else {
+        missingValue = 0L;
+      }
+
+      for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+        final NumericDocValues values = valuesProvider.get(readers.get(readerIndex));
+
+        providers[readerIndex] = docID -> {
+          if (values.advanceExact(docID)) {
+            return values.longValue();
+          } else {
+            return missingValue;
+          }
+        };
+      }
+      return providers;
+    }
+
+    @Override
+    public DocComparator getDocComparator(LeafReader reader, int maxDoc) throws IOException {
+      final NumericDocValues dvs = valuesProvider.get(reader);
+      int[] values = new int[maxDoc];
+      if (this.missingValue != null) {
+        Arrays.fill(values, this.missingValue);
+      }
+      while (true) {
+        int docID = dvs.nextDoc();
+        if (docID == NO_MORE_DOCS) {
+          break;
+        }
+        values[docID] = (int) dvs.longValue();
+      }
+
+      return (docID1, docID2) -> reverseMul * Integer.compare(values[docID1], values[docID2]);
+    }
+
+    @Override
+    public String getProviderName() {
+      return providerName;
+    }
+  }
+
+  /**
+   * Sorts documents based on long values from a NumericDocValues instance
+   */
+  final class LongSorter implements IndexSorter {
+
+    private final String providerName;
+    private final Long missingValue;
+    private final int reverseMul;
+    private final NumericDocValuesProvider valuesProvider;
+
+    /** Creates a new LongSorter */
+    public LongSorter(String providerName, Long missingValue, boolean reverse, NumericDocValuesProvider valuesProvider) {
+      this.providerName = providerName;
+      this.missingValue = missingValue;
+      this.reverseMul = reverse ? -1 : 1;
+      this.valuesProvider = valuesProvider;
+    }
+
+    @Override
+    public ComparableProvider[] getComparableProviders(List<? extends LeafReader> readers) throws IOException {
+      ComparableProvider[] providers = new ComparableProvider[readers.size()];
+      final long missingValue;
+      if (this.missingValue != null) {
+        missingValue = this.missingValue;
+      } else {
+        missingValue = 0L;
+      }
+
+      for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+        final NumericDocValues values = valuesProvider.get(readers.get(readerIndex));
+
+        providers[readerIndex] = docID -> {
+          if (values.advanceExact(docID)) {
+            return values.longValue();
+          } else {
+            return missingValue;
+          }
+        };
+      }
+      return providers;
+    }
+
+    @Override
+    public DocComparator getDocComparator(LeafReader reader, int maxDoc) throws IOException {
+      final NumericDocValues dvs = valuesProvider.get(reader);
+      long[] values = new long[maxDoc];
+      if (this.missingValue != null) {
+        Arrays.fill(values, this.missingValue);
+      }
+      while (true) {
+        int docID = dvs.nextDoc();
+        if (docID == NO_MORE_DOCS) {
+          break;
+        }
+        values[docID] = dvs.longValue();
+      }
+
+      return (docID1, docID2) -> reverseMul * Long.compare(values[docID1], values[docID2]);
+    }
+
+    @Override
+    public String getProviderName() {
+      return providerName;
+    }
+  }
+
+  /**
+   * Sorts documents based on float values from a NumericDocValues instance
+   */
+  final class FloatSorter implements IndexSorter {
+
+    private final String providerName;
+    private final Float missingValue;
+    private final int reverseMul;
+    private final NumericDocValuesProvider valuesProvider;
+
+    /** Creates a new FloatSorter */
+    public FloatSorter(String providerName, Float missingValue, boolean reverse, NumericDocValuesProvider valuesProvider) {
+      this.providerName = providerName;
+      this.missingValue = missingValue;
+      this.reverseMul = reverse ? -1 : 1;
+      this.valuesProvider = valuesProvider;
+    }
+
+    @Override
+    public ComparableProvider[] getComparableProviders(List<? extends LeafReader> readers) throws IOException {
+      ComparableProvider[] providers = new ComparableProvider[readers.size()];
+      final float missingValue;
+      if (this.missingValue != null) {
+        missingValue = this.missingValue;
+      } else {
+        missingValue = 0.0f;
+      }
+
+      for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+        final NumericDocValues values = valuesProvider.get(readers.get(readerIndex));
+
+        providers[readerIndex] = docID -> {
+          float value = missingValue;
+          if (values.advanceExact(docID)) {
+            value = Float.intBitsToFloat((int) values.longValue());
+          }
+          return NumericUtils.floatToSortableInt(value);
+        };
+      }
+      return providers;
+    }
+
+    @Override
+    public DocComparator getDocComparator(LeafReader reader, int maxDoc) throws IOException {
+      final NumericDocValues dvs = valuesProvider.get(reader);
+      float[] values = new float[maxDoc];
+      if (this.missingValue != null) {
+        Arrays.fill(values, this.missingValue);
+      }
+      while (true) {
+        int docID = dvs.nextDoc();
+        if (docID == NO_MORE_DOCS) {
+          break;
+        }
+        values[docID] = Float.intBitsToFloat((int) dvs.longValue());
+      }
+
+      return (docID1, docID2) -> reverseMul * Float.compare(values[docID1], values[docID2]);
+    }
+
+    @Override
+    public String getProviderName() {
+      return providerName;
+    }
+  }
+
+  /**
+   * Sorts documents based on double values from a NumericDocValues instance
+   */
+  final class DoubleSorter implements IndexSorter {
+
+    private final String providerName;
+    private final Double missingValue;
+    private final int reverseMul;
+    private final NumericDocValuesProvider valuesProvider;
+
+    /** Creates a new DoubleSorter */
+    public DoubleSorter(String providerName, Double missingValue, boolean reverse, NumericDocValuesProvider valuesProvider) {
+      this.providerName = providerName;
+      this.missingValue = missingValue;
+      this.reverseMul = reverse ? -1 : 1;
+      this.valuesProvider = valuesProvider;
+    }
+
+    @Override
+    public ComparableProvider[] getComparableProviders(List<? extends LeafReader> readers) throws IOException {
+      ComparableProvider[] providers = new ComparableProvider[readers.size()];
+      final double missingValue;
+      if (this.missingValue != null) {
+        missingValue = this.missingValue;
+      } else {
+        missingValue = 0.0f;
+      }
+
+      for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+        final NumericDocValues values = valuesProvider.get(readers.get(readerIndex));
+
+        providers[readerIndex] = docID -> {
+          double value = missingValue;
+          if (values.advanceExact(docID)) {
+            value = Double.longBitsToDouble(values.longValue());
+          }
+          return NumericUtils.doubleToSortableLong(value);
+        };
+      }
+      return providers;
+    }
+
+    @Override
+    public DocComparator getDocComparator(LeafReader reader, int maxDoc) throws IOException {
+      final NumericDocValues dvs = valuesProvider.get(reader);
+      double[] values = new double[maxDoc];
+      if (missingValue != null) {
+        Arrays.fill(values, missingValue);
+      }
+      while (true) {
+        int docID = dvs.nextDoc();
+        if (docID == NO_MORE_DOCS) {
+          break;
+        }
+        values[docID] = Double.longBitsToDouble(dvs.longValue());
+      }
+
+      return (docID1, docID2) -> reverseMul * Double.compare(values[docID1], values[docID2]);
+    }
+
+    @Override
+    public String getProviderName() {
+      return providerName;
+    }
+  }
+
+  /**
+   * Sorts documents based on terms from a SortedDocValues instance
+   */
+  final class StringSorter implements IndexSorter {
+
+    private final String providerName;
+    private final Object missingValue;
+    private final int reverseMul;
+    private final SortedDocValuesProvider valuesProvider;
+
+    /** Creates a new StringSorter */
+    public StringSorter(String providerName, Object missingValue, boolean reverse, SortedDocValuesProvider valuesProvider) {
+      this.providerName = providerName;
+      this.missingValue = missingValue;
+      this.reverseMul = reverse ? -1 : 1;
+      this.valuesProvider = valuesProvider;
+    }
+
+    @Override
+    public ComparableProvider[] getComparableProviders(List<? extends LeafReader> readers) throws IOException {
+      final ComparableProvider[] providers = new ComparableProvider[readers.size()];
+      final SortedDocValues[] values = new SortedDocValues[readers.size()];
+      for(int i=0;i<readers.size();i++) {
+        final SortedDocValues sorted = valuesProvider.get(readers.get(i));
+        values[i] = sorted;
+      }
+      OrdinalMap ordinalMap = OrdinalMap.build(null, values, PackedInts.DEFAULT);
+      final int missingOrd;
+      if (missingValue == SortField.STRING_LAST) {
+        missingOrd = Integer.MAX_VALUE;
+      } else {
+        missingOrd = Integer.MIN_VALUE;
+      }
+
+      for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+        final SortedDocValues readerValues = values[readerIndex];
+        final LongValues globalOrds = ordinalMap.getGlobalOrds(readerIndex);
+        providers[readerIndex] = docID -> {
+          if (readerValues.advanceExact(docID)) {
+            // translate segment's ord to global ord space:
+            return globalOrds.get(readerValues.ordValue());
+          } else {
+            return missingOrd;
+          }
+        };
+      }
+      return providers;
+    }
+
+    @Override
+    public DocComparator getDocComparator(LeafReader reader, int maxDoc) throws IOException {
+      final SortedDocValues sorted = valuesProvider.get(reader);
+      final int missingOrd;
+      if (missingValue == SortField.STRING_LAST) {
+        missingOrd = Integer.MAX_VALUE;
+      } else {
+        missingOrd = Integer.MIN_VALUE;
+      }
+
+      final int[] ords = new int[maxDoc];
+      Arrays.fill(ords, missingOrd);
+      int docID;
+      while ((docID = sorted.nextDoc()) != NO_MORE_DOCS) {
+        ords[docID] = sorted.ordValue();
+      }
+
+      return (docID1, docID2) -> reverseMul * Integer.compare(ords[docID1], ords[docID2]);
+    }
+
+    @Override
+    public String getProviderName() {
+      return providerName;
+    }
+  }
+
+}
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
index 4cdc9c0..26e7e3d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import java.io.PrintStream;
 import java.util.Arrays;
-import java.util.EnumSet;
 import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -32,9 +31,9 @@ import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.PrintStreamInfoStream;
+import org.apache.lucene.util.SetOnce;
 import org.apache.lucene.util.SetOnce.AlreadySetException;
 import org.apache.lucene.util.Version;
-import org.apache.lucene.util.SetOnce;
 
 /**
  * Holds all the configuration that is used to create an {@link IndexWriter}.
@@ -460,21 +459,13 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
     return this;
   }
 
-  /** We only allow sorting on these types */
-  private static final EnumSet<SortField.Type> ALLOWED_INDEX_SORT_TYPES = EnumSet.of(SortField.Type.STRING,
-                                                                                     SortField.Type.LONG,
-                                                                                     SortField.Type.INT,
-                                                                                     SortField.Type.DOUBLE,
-                                                                                     SortField.Type.FLOAT);
-
   /**
    * Set the {@link Sort} order to use for all (flushed and merged) segments.
    */
   public IndexWriterConfig setIndexSort(Sort sort) {
-    for(SortField sortField : sort.getSort()) {
-      final SortField.Type sortType = Sorter.getSortFieldType(sortField);
-      if (ALLOWED_INDEX_SORT_TYPES.contains(sortType) == false) {
-        throw new IllegalArgumentException("invalid SortField type: must be one of " + ALLOWED_INDEX_SORT_TYPES + " but got: " + sortField);
+    for (SortField sortField : sort.getSort()) {
+      if (sortField.getIndexSorter() == null) {
+        throw new IllegalArgumentException("Cannot sort index with sort field " + sortField);
       }
     }
     this.indexSort = sort;
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
index 2348701..35d1441 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
@@ -24,8 +24,6 @@ import org.apache.lucene.index.MergeState.DocMap;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.LongValues;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
@@ -41,10 +39,14 @@ final class MultiSorter {
     // TODO: optimize if only 1 reader is incoming, though that's a rare case
 
     SortField fields[] = sort.getSort();
-    final ComparableProvider[][] comparables = new ComparableProvider[fields.length][];
+    final IndexSorter.ComparableProvider[][] comparables = new IndexSorter.ComparableProvider[fields.length][];
     final int[] reverseMuls = new int[fields.length];
     for(int i=0;i<fields.length;i++) {
-      comparables[i] = getComparableProviders(readers, fields[i]);
+      IndexSorter sorter = fields[i].getIndexSorter();
+      if (sorter == null) {
+        throw new IllegalArgumentException("Cannot use sort field " + fields[i] + " for index sorting");
+      }
+      comparables[i] = sorter.getComparableProviders(readers);
       reverseMuls[i] = fields[i].getReverse() ? -1 : 1;
     }
     int leafCount = readers.size();
@@ -142,138 +144,4 @@ final class MultiSorter {
       this.valuesAsComparableLongs = new long[numComparables];
     }
   }
-
-  /** Returns a long so that the natural ordering of long values matches the
-   *  ordering of doc IDs for the given comparator. */
-  private interface ComparableProvider {
-    long getAsComparableLong(int docID) throws IOException;
-  }
-
-  /** Returns {@code ComparableProvider}s for the provided readers to represent the requested {@link SortField} sort order. */
-  private static ComparableProvider[] getComparableProviders(List<CodecReader> readers, SortField sortField) throws IOException {
-
-    ComparableProvider[] providers = new ComparableProvider[readers.size()];
-    final SortField.Type sortType = Sorter.getSortFieldType(sortField);
-
-    switch(sortType) {
-
-    case STRING:
-      {
-        // this uses the efficient segment-local ordinal map:
-        final SortedDocValues[] values = new SortedDocValues[readers.size()];
-        for(int i=0;i<readers.size();i++) {
-          final SortedDocValues sorted = Sorter.getOrWrapSorted(readers.get(i), sortField);
-          values[i] = sorted;
-        }
-        OrdinalMap ordinalMap = OrdinalMap.build(null, values, PackedInts.DEFAULT);
-        final int missingOrd;
-        if (sortField.getMissingValue() == SortField.STRING_LAST) {
-          missingOrd = Integer.MAX_VALUE;
-        } else {
-          missingOrd = Integer.MIN_VALUE;
-        }
-
-        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
-          final SortedDocValues readerValues = values[readerIndex];
-          final LongValues globalOrds = ordinalMap.getGlobalOrds(readerIndex);
-          providers[readerIndex] = new ComparableProvider() {
-              @Override
-              public long getAsComparableLong(int docID) throws IOException {
-                if (readerValues.advanceExact(docID)) {
-                  // translate segment's ord to global ord space:
-                  return globalOrds.get(readerValues.ordValue());
-                } else {
-                  return missingOrd;
-                }
-              }
-            };
-        }
-      }
-      break;
-
-    case LONG:
-    case INT:
-      {
-        final long missingValue;
-        if (sortField.getMissingValue() != null) {
-          missingValue = ((Number) sortField.getMissingValue()).longValue();
-        } else {
-          missingValue = 0L;
-        }
-
-        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
-          final NumericDocValues values = Sorter.getOrWrapNumeric(readers.get(readerIndex), sortField);
-
-          providers[readerIndex] = new ComparableProvider() {
-              @Override
-              public long getAsComparableLong(int docID) throws IOException {
-                if (values.advanceExact(docID)) {
-                  return values.longValue();
-                } else {
-                  return missingValue;
-                }
-              }
-            };
-        }
-      }
-      break;
-
-    case DOUBLE:
-      {
-        final double missingValue;
-        if (sortField.getMissingValue() != null) {
-          missingValue = (Double) sortField.getMissingValue();
-        } else {
-          missingValue = 0.0;
-        }
-
-        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
-          final NumericDocValues values = Sorter.getOrWrapNumeric(readers.get(readerIndex), sortField);
-
-          providers[readerIndex] = new ComparableProvider() {
-              @Override
-              public long getAsComparableLong(int docID) throws IOException {
-                double value = missingValue;
-                if (values.advanceExact(docID)) {
-                  value = Double.longBitsToDouble(values.longValue());
-                }
-                return NumericUtils.doubleToSortableLong(value);
-              }
-            };
-        }
-      }
-      break;
-
-    case FLOAT:
-      {
-        final float missingValue;
-        if (sortField.getMissingValue() != null) {
-          missingValue = (Float) sortField.getMissingValue();
-        } else {
-          missingValue = 0.0f;
-        }
-
-        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
-          final NumericDocValues values = Sorter.getOrWrapNumeric(readers.get(readerIndex), sortField);
-
-          providers[readerIndex] = new ComparableProvider() {
-              @Override
-              public long getAsComparableLong(int docID) throws IOException {
-                float value = missingValue;
-                if (values.advanceExact(docID)) {
-                  value = Float.intBitsToFloat((int) values.longValue());
-                }
-                return NumericUtils.floatToSortableInt(value);
-              }
-            };
-        }
-      }
-      break;
-
-    default:
-      throw new IllegalArgumentException("unhandled SortField.getType()=" + sortField.getType());
-    }
-
-    return providers;
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
index 980849f..87de98f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.packed.PackedInts;
@@ -31,7 +30,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 /** Buffers up pending long per doc, then flushes when
  *  segment flushes. */
-class NumericDocValuesWriter extends DocValuesWriter {
+class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
 
   private PackedLongValues.Builder pending;
   private PackedLongValues finalValues;
@@ -70,21 +69,11 @@ class NumericDocValuesWriter extends DocValuesWriter {
   }
 
   @Override
-  public void finish(int maxDoc) {
-  }
-
-  @Override
-  Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
-    assert finalValues == null;
-    finalValues = pending.build();
-    final BufferedNumericDocValues docValues =
-        new BufferedNumericDocValues(finalValues, docsWithField.iterator());
-    return Sorter.getDocComparator(maxDoc, sortField, () -> null, () -> docValues);
-  }
-
-  @Override
-  DocIdSetIterator getDocIdSet() {
-    return docsWithField.iterator();
+  NumericDocValues getDocValues() {
+    if (finalValues == null) {
+      finalValues = pending.build();
+    }
+    return new BufferedNumericDocValues(finalValues, docsWithField.iterator());
   }
 
   static SortingLeafReader.CachedNumericDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues) throws IOException {
@@ -104,16 +93,12 @@ class NumericDocValuesWriter extends DocValuesWriter {
 
   @Override
   public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
-    final PackedLongValues values;
     if (finalValues == null) {
-      values = pending.build();
-    } else {
-      values = finalValues;
+      finalValues = pending.build();
     }
-
     final SortingLeafReader.CachedNumericDVs sorted;
     if (sortMap != null) {
-      NumericDocValues oldValues = new BufferedNumericDocValues(values, docsWithField.iterator());
+      NumericDocValues oldValues = new BufferedNumericDocValues(finalValues, docsWithField.iterator());
       sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap, oldValues);
     } else {
       sorted = null;
@@ -127,7 +112,7 @@ class NumericDocValuesWriter extends DocValuesWriter {
                                      throw new IllegalArgumentException("wrong fieldInfo");
                                    }
                                    if (sorted == null) {
-                                     return new BufferedNumericDocValues(values, docsWithField.iterator());
+                                     return new BufferedNumericDocValues(finalValues, docsWithField.iterator());
                                    } else {
                                      return new SortingLeafReader.SortingNumericDocValues(sorted);
                                    }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortFieldProvider.java b/lucene/core/src/java/org/apache/lucene/index/SortFieldProvider.java
new file mode 100644
index 0000000..290decd
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/SortFieldProvider.java
@@ -0,0 +1,118 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.NamedSPILoader;
+
+/**
+ * Reads/Writes a named SortField from a segment info file, used to record index sorts
+ */
+public abstract class SortFieldProvider implements NamedSPILoader.NamedSPI {
+
+  private static class Holder {
+    private static final NamedSPILoader<SortFieldProvider> LOADER = new NamedSPILoader<>(SortFieldProvider.class);
+
+    static NamedSPILoader<SortFieldProvider> getLoader() {
+      if (LOADER == null) {
+        throw new IllegalStateException("You tried to lookup a SortFieldProvider by name before all SortFieldProviders could be initialized. "+
+            "This likely happens if you call SortFieldProvider#forName from a SortFieldProviders's ctor.");
+      }
+      return LOADER;
+    }
+  }
+
+  /**
+   * Looks up a SortFieldProvider by name
+   */
+  public static SortFieldProvider forName(String name) {
+    return Holder.getLoader().lookup(name);
+  }
+
+  /**
+   * Lists all available SortFieldProviders
+   */
+  public static Set<String> availableSortFieldProviders() {
+    return Holder.getLoader().availableServices();
+  }
+
+  /**
+   * Reloads the SortFieldProvider list from the given {@link ClassLoader}.
+   * Changes to the list are visible after the method ends, all
+   * iterators ({@link #availableSortFieldProviders()} ()},...) stay consistent.
+   *
+   * <p><b>NOTE:</b> Only new SortFieldProviders are added, existing ones are
+   * never removed or replaced.
+   *
+   * <p><em>This method is expensive and should only be called for discovery
+   * of new SortFieldProviders on the given classpath/classloader!</em>
+   */
+  public static void reloadSortFieldProviders(ClassLoader classLoader) {
+    Holder.getLoader().reload(classLoader);
+  }
+
+  /**
+   * Writes a SortField to a DataOutput
+   */
+  public static void write(SortField sf, DataOutput output) throws IOException {
+    IndexSorter sorter = sf.getIndexSorter();
+    if (sorter == null) {
+      throw new IllegalArgumentException("Cannot serialize sort field " + sf);
+    }
+    SortFieldProvider provider = SortFieldProvider.forName(sorter.getProviderName());
+    provider.writeSortField(sf, output);
+  }
+
+  /** The name this SortFieldProvider is registered under */
+  protected final String name;
+
+  /**
+   * Creates a new SortFieldProvider.
+   * <p>
+   * The provided name will be written into the index segment: in order to
+   * for the segment to be read this class should be registered with Java's
+   * SPI mechanism (registered in META-INF/ of your jar file, etc).
+   * @param name must be all ascii alphanumeric, and less than 128 characters in length.
+   */
+  protected SortFieldProvider(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Reads a SortField from serialized bytes
+   */
+  public abstract SortField readSortField(DataInput in) throws IOException;
+
+  /**
+   * Writes a SortField to a DataOutput
+   *
+   * This is used to record index sort information in segment headers
+   */
+  public abstract void writeSortField(SortField sf, DataOutput out) throws IOException;
+
+}
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
index 86d0f0b..2252f00 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
@@ -21,7 +21,6 @@ import java.util.Arrays;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
@@ -35,7 +34,7 @@ import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
 /** Buffers up pending byte[] per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
-class SortedDocValuesWriter extends DocValuesWriter {
+class SortedDocValuesWriter extends DocValuesWriter<SortedDocValues> {
   final BytesRefHash hash;
   private PackedLongValues.Builder pending;
   private DocsWithFieldSet docsWithField;
@@ -79,11 +78,6 @@ class SortedDocValuesWriter extends DocValuesWriter {
     lastDocID = docID;
   }
 
-  @Override
-  public void finish(int maxDoc) {
-    updateBytesUsed();
-  }
-
   private void addOneValue(BytesRef value) {
     int termID = hash.add(value);
     if (termID < 0) {
@@ -107,20 +101,20 @@ class SortedDocValuesWriter extends DocValuesWriter {
   }
 
   @Override
-  Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
-    assert sortField.getType().equals(SortField.Type.STRING);
-    assert finalSortedValues == null && finalOrdMap == null &&finalOrds == null;
+  SortedDocValues getDocValues() {
     int valueCount = hash.size();
-    finalSortedValues = hash.sort();
-    finalOrds = pending.build();
-    finalOrdMap = new int[valueCount];
+    if (finalSortedValues == null) {
+      updateBytesUsed();
+      assert finalOrdMap == null && finalOrds == null;
+      finalSortedValues = hash.sort();
+      finalOrds = pending.build();
+      finalOrdMap = new int[valueCount];
+    }
     for (int ord = 0; ord < valueCount; ord++) {
       finalOrdMap[finalSortedValues[ord]] = ord;
     }
-    final SortedDocValues docValues =
-        new BufferedSortedDocValues(hash, valueCount, finalOrds, finalSortedValues, finalOrdMap,
+    return new BufferedSortedDocValues(hash, valueCount, finalOrds, finalSortedValues, finalOrdMap,
             docsWithField.iterator());
-    return Sorter.getDocComparator(maxDoc, sortField, () -> docValues, () -> null);
   }
 
   private int[] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedDocValues oldValues) throws IOException {
@@ -137,26 +131,20 @@ class SortedDocValuesWriter extends DocValuesWriter {
   @Override
   public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
     final int valueCount = hash.size();
-    final PackedLongValues ords;
-    final int[] sortedValues;
-    final int[] ordMap;
     if (finalOrds == null) {
-      sortedValues = hash.sort();
-      ords = pending.build();
-      ordMap = new int[valueCount];
+      updateBytesUsed();
+      finalSortedValues = hash.sort();
+      finalOrds = pending.build();
+      finalOrdMap = new int[valueCount];
       for (int ord = 0; ord < valueCount; ord++) {
-        ordMap[sortedValues[ord]] = ord;
+        finalOrdMap[finalSortedValues[ord]] = ord;
       }
-    } else {
-      sortedValues = finalSortedValues;
-      ords = finalOrds;
-      ordMap = finalOrdMap;
     }
 
     final int[] sorted;
     if (sortMap != null) {
       sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap,
-          new BufferedSortedDocValues(hash, valueCount, ords, sortedValues, ordMap, docsWithField.iterator()));
+          new BufferedSortedDocValues(hash, valueCount, finalOrds, finalSortedValues, finalOrdMap, docsWithField.iterator()));
     } else {
       sorted = null;
     }
@@ -168,7 +156,7 @@ class SortedDocValuesWriter extends DocValuesWriter {
                                     throw new IllegalArgumentException("wrong fieldInfo");
                                   }
                                   final SortedDocValues buf =
-                                      new BufferedSortedDocValues(hash, valueCount, ords, sortedValues, ordMap, docsWithField.iterator());
+                                      new BufferedSortedDocValues(hash, valueCount, finalOrds, finalSortedValues, finalOrdMap, docsWithField.iterator());
                                   if (sorted == null) {
                                    return buf;
                                   }
@@ -245,8 +233,4 @@ class SortedDocValuesWriter extends DocValuesWriter {
     }
   }
 
-  @Override
-  DocIdSetIterator getDocIdSet() {
-    return docsWithField.iterator();
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
index bdc65cc..83c394f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
@@ -22,9 +22,6 @@ import java.util.Arrays;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSelector;
-import org.apache.lucene.search.SortedNumericSortField;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -34,7 +31,7 @@ import org.apache.lucene.util.packed.PackedLongValues;
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 /** Buffers up pending long[] per doc, sorts, then flushes when segment flushes. */
-class SortedNumericDocValuesWriter extends DocValuesWriter {
+class SortedNumericDocValuesWriter extends DocValuesWriter<SortedNumericDocValues> {
   private PackedLongValues.Builder pending; // stream of all values
   private PackedLongValues.Builder pendingCounts; // count of values per doc
   private DocsWithFieldSet docsWithField;
@@ -85,11 +82,6 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
     docsWithField.add(currentDoc);
   }
 
-  @Override
-  public void finish(int maxDoc) {
-    finishCurrentDoc();
-  }
-
   private void addOneValue(long value) {
     if (currentUpto == currentValues.length) {
       currentValues = ArrayUtil.grow(currentValues, currentValues.length+1);
@@ -106,16 +98,14 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
   }
 
   @Override
-  Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
-    assert sortField instanceof SortedNumericSortField;
-    assert finalValues == null && finalValuesCount == null;
-    finalValues = pending.build();
-    finalValuesCount = pendingCounts.build();
-    final SortedNumericDocValues docValues =
-        new BufferedSortedNumericDocValues(finalValues, finalValuesCount, docsWithField.iterator());
-    SortedNumericSortField sf = (SortedNumericSortField) sortField;
-    return Sorter.getDocComparator(maxDoc, sf, () -> null,
-        () -> SortedNumericSelector.wrap(docValues, sf.getSelector(), sf.getNumericType()));
+  SortedNumericDocValues getDocValues() {
+    if (finalValues == null) {
+      assert finalValuesCount == null;
+      finishCurrentDoc();
+      finalValues = pending.build();
+      finalValuesCount = pendingCounts.build();
+    }
+    return new BufferedSortedNumericDocValues(finalValues, finalValuesCount, docsWithField.iterator());
   }
 
   private long[][] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedNumericDocValues oldValues) throws IOException {
@@ -137,6 +127,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
     final PackedLongValues values;
     final PackedLongValues valueCounts;
     if (finalValues == null) {
+      finishCurrentDoc();
       values = pending.build();
       valueCounts = pendingCounts.build();
     } else {
@@ -232,8 +223,4 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
     }
   }
 
-  @Override
-  DocIdSetIterator getDocIdSet() {
-    return docsWithField.iterator();
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
index 71a14a5..022b17d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
@@ -21,9 +21,6 @@ import java.util.Arrays;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedSetSelector;
-import org.apache.lucene.search.SortedSetSortField;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
@@ -39,7 +36,7 @@ import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
 /** Buffers up pending byte[]s per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
-class SortedSetDocValuesWriter extends DocValuesWriter {
+class SortedSetDocValuesWriter extends DocValuesWriter<SortedSetDocValues> {
   final BytesRefHash hash;
   private PackedLongValues.Builder pending; // stream of all termIDs
   private PackedLongValues.Builder pendingCounts; // termIDs per doc
@@ -115,11 +112,6 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
     docsWithField.add(currentDoc);
   }
 
-  @Override
-  public void finish(int maxDoc) {
-    finishCurrentDoc();
-  }
-
   private void addOneValue(BytesRef value) {
     int termID = hash.add(value);
     if (termID < 0) {
@@ -170,22 +162,20 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
   }
 
   @Override
-  Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
-    assert sortField instanceof SortedSetSortField;
-    assert finalOrds == null && finalOrdCounts == null && finalSortedValues == null && finalOrdMap == null;
-    int valueCount = hash.size();
-    finalOrds = pending.build();
-    finalOrdCounts = pendingCounts.build();
-    finalSortedValues = hash.sort();
-    finalOrdMap = new int[valueCount];
-    for (int ord = 0; ord < valueCount; ord++) {
+  SortedSetDocValues getDocValues() {
+    if (finalOrds == null) {
+      assert finalOrdCounts == null && finalSortedValues == null && finalOrdMap == null;
+      finishCurrentDoc();
+      int valueCount = hash.size();
+      finalOrds = pending.build();
+      finalOrdCounts = pendingCounts.build();
+      finalSortedValues = hash.sort();
+      finalOrdMap = new int[valueCount];
+    }
+    for (int ord = 0; ord < finalOrdMap.length; ord++) {
       finalOrdMap[finalSortedValues[ord]] = ord;
     }
-
-    SortedSetSortField sf = (SortedSetSortField) sortField;
-    final SortedSetDocValues dvs =
-        new BufferedSortedSetDocValues(finalSortedValues, finalOrdMap, hash, finalOrds, finalOrdCounts, maxCount, docsWithField.iterator());
-    return Sorter.getDocComparator(maxDoc, sf, () -> SortedSetSelector.wrap(dvs, sf.getSelector()), () -> null);
+    return new BufferedSortedSetDocValues(finalSortedValues, finalOrdMap, hash, finalOrds, finalOrdCounts, maxCount, docsWithField.iterator());
   }
 
   @Override
@@ -196,7 +186,9 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
     final int[] sortedValues;
     final int[] ordMap;
 
-    if (finalOrdCounts == null) {
+    if (finalOrds == null) {
+      assert finalOrdCounts == null && finalSortedValues == null && finalOrdMap == null;
+      finishCurrentDoc();
       ords = pending.build();
       ordCounts = pendingCounts.build();
       sortedValues = hash.sort();
@@ -315,8 +307,5 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
       return scratch;
     }
   }
-  @Override
-  DocIdSetIterator getDocIdSet() {
-    return docsWithField.iterator();
-  }
+
 }
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 5f43c5a..c860569 100644
--- a/lucene/core/src/java/org/apache/lucene/index/Sorter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/Sorter.java
@@ -17,22 +17,13 @@
 package org.apache.lucene.index;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Comparator;
 
-import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedNumericSelector;
-import org.apache.lucene.search.SortedNumericSortField;
-import org.apache.lucene.search.SortedSetSelector;
-import org.apache.lucene.search.SortedSetSortField;
 import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
-import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
-
 /**
  * Sorts documents of a given index by returning a permutation on the document
  * IDs.
@@ -84,21 +75,13 @@ final class Sorter {
     return true;
   }
 
-  /** A comparator of doc IDs. */
-  static abstract class DocComparator {
-
-    /** Compare docID1 against docID2. The contract for the return value is the
-     *  same as {@link Comparator#compare(Object, Object)}. */
-    public abstract int compare(int docID1, int docID2);
-  }
-
   private static final class DocValueSorter extends TimSorter {
     
     private final int[] docs;
-    private final Sorter.DocComparator comparator;
+    private final IndexSorter.DocComparator comparator;
     private final int[] tmp;
     
-    DocValueSorter(int[] docs, Sorter.DocComparator comparator) {
+    DocValueSorter(int[] docs, IndexSorter.DocComparator comparator) {
       super(docs.length / 64);
       this.docs = docs;
       this.comparator = comparator;
@@ -139,7 +122,7 @@ final class Sorter {
   }
 
   /** Computes the old-to-new permutation over the given comparator. */
-  private static Sorter.DocMap sort(final int maxDoc, DocComparator comparator) {
+  private static Sorter.DocMap sort(final int maxDoc, IndexSorter.DocComparator comparator) {
     // check if the index is sorted
     boolean sorted = true;
     for (int i = 1; i < maxDoc; ++i) {
@@ -202,196 +185,10 @@ final class Sorter {
     };
   }
 
-  /** Returns the native sort type for {@link SortedSetSortField} and {@link SortedNumericSortField},
-   * {@link SortField#getType()} otherwise */
-  static SortField.Type getSortFieldType(SortField sortField) {
-    if (sortField instanceof SortedSetSortField) {
-      return SortField.Type.STRING;
-    } else if (sortField instanceof SortedNumericSortField) {
-      return ((SortedNumericSortField) sortField).getNumericType();
-    } else {
-      return sortField.getType();
-    }
-  }
-
-  /** Wraps a {@link SortedNumericDocValues} as a single-valued view if the field is an instance of {@link SortedNumericSortField},
-   * returns {@link NumericDocValues} for the field otherwise. */
-  static NumericDocValues getOrWrapNumeric(LeafReader reader, SortField sortField) throws IOException {
-    if (sortField instanceof SortedNumericSortField) {
-      SortedNumericSortField sf = (SortedNumericSortField) sortField;
-      return SortedNumericSelector.wrap(DocValues.getSortedNumeric(reader, sf.getField()), sf.getSelector(), sf.getNumericType());
-    } else {
-      return DocValues.getNumeric(reader, sortField.getField());
-    }
-  }
-
-  /** Wraps a {@link SortedSetDocValues} as a single-valued view if the field is an instance of {@link SortedSetSortField},
-   * returns {@link SortedDocValues} for the field otherwise. */
-  static SortedDocValues getOrWrapSorted(LeafReader reader, SortField sortField) throws IOException {
-    if (sortField instanceof SortedSetSortField) {
-      SortedSetSortField sf = (SortedSetSortField) sortField;
-      return SortedSetSelector.wrap(DocValues.getSortedSet(reader, sf.getField()), sf.getSelector());
-    } else {
-      return DocValues.getSorted(reader, sortField.getField());
-    }
-  }
-
-  static DocComparator getDocComparator(LeafReader reader, SortField sortField) throws IOException {
-    return getDocComparator(reader.maxDoc(), sortField,
-        () -> getOrWrapSorted(reader, sortField),
-        () -> getOrWrapNumeric(reader, sortField));
-  }
-
-  interface NumericDocValuesSupplier {
-    NumericDocValues get() throws IOException;
-  }
-
-  interface SortedDocValuesSupplier {
-    SortedDocValues get() throws IOException;
-  }
-
-  /** We cannot use the {@link FieldComparator} API because that API requires that you send it docIDs in order.  Note that this API
-   *  allocates arrays[maxDoc] to hold the native values needed for comparison, but 1) they are transient (only alive while sorting this one
-   *  segment), and 2) in the typical index sorting case, they are only used to sort newly flushed segments, which will be smaller than
-   *  merged segments.  */
-  static DocComparator getDocComparator(int maxDoc,
-                                        SortField sortField,
-                                        SortedDocValuesSupplier sortedProvider,
-                                        NumericDocValuesSupplier numericProvider) throws IOException {
-
-    final int reverseMul = sortField.getReverse() ? -1 : 1;
-    final SortField.Type sortType = getSortFieldType(sortField);
-
-    switch(sortType) {
-
-      case STRING:
-      {
-        final SortedDocValues sorted = sortedProvider.get();
-        final int missingOrd;
-        if (sortField.getMissingValue() == SortField.STRING_LAST) {
-          missingOrd = Integer.MAX_VALUE;
-        } else {
-          missingOrd = Integer.MIN_VALUE;
-        }
-
-        final int[] ords = new int[maxDoc];
-        Arrays.fill(ords, missingOrd);
-        int docID;
-        while ((docID = sorted.nextDoc()) != NO_MORE_DOCS) {
-          ords[docID] = sorted.ordValue();
-        }
-
-        return new DocComparator() {
-          @Override
-          public int compare(int docID1, int docID2) {
-            return reverseMul * Integer.compare(ords[docID1], ords[docID2]);
-          }
-        };
-      }
-
-      case LONG:
-      {
-        final NumericDocValues dvs = numericProvider.get();
-        long[] values = new long[maxDoc];
-        if (sortField.getMissingValue() != null) {
-          Arrays.fill(values, (Long) sortField.getMissingValue());
-        }
-        while (true) {
-          int docID = dvs.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            break;
-          }
-          values[docID] = dvs.longValue();
-        }
-
-        return new DocComparator() {
-          @Override
-          public int compare(int docID1, int docID2) {
-            return reverseMul * Long.compare(values[docID1], values[docID2]);
-          }
-        };
-      }
-
-      case INT:
-      {
-        final NumericDocValues dvs = numericProvider.get();
-        int[] values = new int[maxDoc];
-        if (sortField.getMissingValue() != null) {
-          Arrays.fill(values, (Integer) sortField.getMissingValue());
-        }
-
-        while (true) {
-          int docID = dvs.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            break;
-          }
-          values[docID] = (int) dvs.longValue();
-        }
-
-        return new DocComparator() {
-          @Override
-          public int compare(int docID1, int docID2) {
-            return reverseMul * Integer.compare(values[docID1], values[docID2]);
-          }
-        };
-      }
-
-      case DOUBLE:
-      {
-        final NumericDocValues dvs = numericProvider.get();
-        double[] values = new double[maxDoc];
-        if (sortField.getMissingValue() != null) {
-          Arrays.fill(values, (Double) sortField.getMissingValue());
-        }
-        while (true) {
-          int docID = dvs.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            break;
-          }
-          values[docID] = Double.longBitsToDouble(dvs.longValue());
-        }
-
-        return new DocComparator() {
-          @Override
-          public int compare(int docID1, int docID2) {
-            return reverseMul * Double.compare(values[docID1], values[docID2]);
-          }
-        };
-      }
-
-      case FLOAT:
-      {
-        final NumericDocValues dvs = numericProvider.get();
-        float[] values = new float[maxDoc];
-        if (sortField.getMissingValue() != null) {
-          Arrays.fill(values, (Float) sortField.getMissingValue());
-        }
-        while (true) {
-          int docID = dvs.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            break;
-          }
-          values[docID] = Float.intBitsToFloat((int) dvs.longValue());
-        }
-
-        return new DocComparator() {
-          @Override
-          public int compare(int docID1, int docID2) {
-            return reverseMul * Float.compare(values[docID1], values[docID2]);
-          }
-        };
-      }
-
-      default:
-        throw new IllegalArgumentException("unhandled SortField.getType()=" + sortField.getType());
-    }
-  }
-
-
   /**
    * Returns a mapping from the old document ID to its new location in the
    * sorted index. Implementations can use the auxiliary
-   * {@link #sort(int, DocComparator)} to compute the old-to-new permutation
+   * {@link #sort(int, IndexSorter.DocComparator)} to compute the old-to-new permutation
    * given a list of documents and their corresponding values.
    * <p>
    * A return value of <code>null</code> is allowed and means that
@@ -401,28 +198,29 @@ final class Sorter {
    * well, they will however be marked as deleted in the sorted view.
    */
   DocMap sort(LeafReader reader) throws IOException {
-    SortField fields[] = sort.getSort();
-    final DocComparator comparators[] = new DocComparator[fields.length];
+    SortField[] fields = sort.getSort();
+    final IndexSorter.DocComparator[] comparators = new IndexSorter.DocComparator[fields.length];
 
     for (int i = 0; i < fields.length; i++) {
-      comparators[i] = getDocComparator(reader, fields[i]);
+      IndexSorter sorter = fields[i].getIndexSorter();
+      if (sorter == null) {
+        throw new IllegalArgumentException("Cannot use sortfield + "  + fields[i] + " to sort indexes");
+      }
+      comparators[i] = sorter.getDocComparator(reader, reader.maxDoc());
     }
     return sort(reader.maxDoc(), comparators);
   }
 
 
-  DocMap sort(int maxDoc, DocComparator[] comparators) throws IOException {
-    final DocComparator comparator = new DocComparator() {
-      @Override
-      public int compare(int docID1, int docID2) {
-        for (int i = 0; i < comparators.length; i++) {
-          int comp = comparators[i].compare(docID1, docID2);
-          if (comp != 0) {
-            return comp;
-          }
+  DocMap sort(int maxDoc, IndexSorter.DocComparator[] comparators) throws IOException {
+    final IndexSorter.DocComparator comparator = (docID1, docID2) -> {
+      for (int i = 0; i < comparators.length; i++) {
+        int comp = comparators[i].compare(docID1, docID2);
+        if (comp != 0) {
+          return comp;
         }
-        return Integer.compare(docID1, docID2); // docid order tiebreak
       }
+      return Integer.compare(docID1, docID2); // docid order tiebreak
     };
 
     return sort(maxDoc, comparator);
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortField.java b/lucene/core/src/java/org/apache/lucene/search/SortField.java
index 2cfae46..7512ec9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortField.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortField.java
@@ -21,7 +21,13 @@ import java.io.IOException;
 import java.util.Comparator;
 import java.util.Objects;
 
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexSorter;
+import org.apache.lucene.index.SortFieldProvider;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
 
 /**
  * Stores information about how to sort documents by terms in an individual
@@ -120,6 +126,106 @@ public class SortField {
     this.reverse = reverse;
   }
 
+  /** A SortFieldProvider for field sorts */
+  public static final class Provider extends SortFieldProvider {
+
+    /** The name this Provider is registered under */
+    public static final String NAME = "SortField";
+
+    /** Creates a new Provider */
+    public Provider() {
+      super(NAME);
+    }
+
+    @Override
+    public SortField readSortField(DataInput in) throws IOException {
+      SortField sf = new SortField(in.readString(), readType(in), in.readInt() == 1);
+      if (in.readInt() == 1) {
+        // missing object
+        switch (sf.type) {
+          case STRING:
+            int missingString = in.readInt();
+            if (missingString == 1) {
+              sf.setMissingValue(STRING_FIRST);
+            }
+            else {
+              sf.setMissingValue(STRING_LAST);
+            }
+            break;
+          case INT:
+            sf.setMissingValue(in.readInt());
+            break;
+          case LONG:
+            sf.setMissingValue(in.readLong());
+            break;
+          case FLOAT:
+            sf.setMissingValue(NumericUtils.sortableIntToFloat(in.readInt()));
+            break;
+          case DOUBLE:
+            sf.setMissingValue(NumericUtils.sortableLongToDouble(in.readLong()));
+            break;
+          default:
+            throw new IllegalArgumentException("Cannot deserialize sort of type " + sf.type);
+        }
+      }
+      return sf;
+    }
+
+    @Override
+    public void writeSortField(SortField sf, DataOutput out) throws IOException {
+      sf.serialize(out);
+    }
+  }
+
+  protected static Type readType(DataInput in) throws IOException {
+    String type = in.readString();
+    try {
+      return Type.valueOf(type);
+    }
+    catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Can't deserialize SortField - unknown type " + type);
+    }
+  }
+
+  private void serialize(DataOutput out) throws IOException {
+    out.writeString(field);
+    out.writeString(type.toString());
+    out.writeInt(reverse ? 1 : 0);
+    if (missingValue == null) {
+      out.writeInt(0);
+    }
+    else {
+      out.writeInt(1);
+      switch (type) {
+        case STRING:
+          if (missingValue == STRING_LAST) {
+            out.writeInt(0);
+          }
+          else if (missingValue == STRING_FIRST) {
+            out.writeInt(1);
+          }
+          else {
+            throw new IllegalArgumentException("Cannot serialize missing value of " + missingValue + " for type STRING");
+          }
+          break;
+        case INT:
+          out.writeInt((int)missingValue);
+          break;
+        case LONG:
+          out.writeLong((long)missingValue);
+          break;
+        case FLOAT:
+          out.writeInt(NumericUtils.floatToSortableInt((float)missingValue));
+          break;
+        case DOUBLE:
+          out.writeLong(NumericUtils.doubleToSortableLong((double)missingValue));
+          break;
+        default:
+          throw new IllegalArgumentException("Cannot serialize SortField of type " + type);
+      }
+    }
+  }
+
   /** Pass this to {@link #setMissingValue} to have missing
    *  string values sort first. */
   public final static Object STRING_FIRST = new Object() {
@@ -392,4 +498,33 @@ public class SortField {
   public boolean needsScores() {
     return type == Type.SCORE;
   }
+
+  /**
+   * Returns an {@link IndexSorter} used for sorting index segments by this SortField.
+   *
+   * If the SortField cannot be used for index sorting (for example, if it uses scores or
+   * other query-dependent values) then this method should return {@code null}
+   *
+   * SortFields that implement this method should also implement a companion
+   * {@link SortFieldProvider} to serialize and deserialize the sort in index segment
+   * headers
+   *
+   * @lucene.experimental
+   */
+  public IndexSorter getIndexSorter() {
+    switch (type) {
+      case STRING:
+        return new IndexSorter.StringSorter(Provider.NAME, missingValue, reverse, reader -> DocValues.getSorted(reader, field));
+      case INT:
+        return new IndexSorter.IntSorter(Provider.NAME, (Integer)missingValue, reverse, reader -> DocValues.getNumeric(reader, field));
+      case LONG:
+        return new IndexSorter.LongSorter(Provider.NAME, (Long)missingValue, reverse, reader -> DocValues.getNumeric(reader, field));
+      case DOUBLE:
+        return new IndexSorter.DoubleSorter(Provider.NAME, (Double)missingValue, reverse, reader -> DocValues.getNumeric(reader, field));
+      case FLOAT:
+        return new IndexSorter.FloatSorter(Provider.NAME, (Float)missingValue, reverse, reader -> DocValues.getNumeric(reader, field));
+      default: return null;
+    }
+  }
+
 }
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
index fff000b..6c5154a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
@@ -20,9 +20,15 @@ package org.apache.lucene.search;
 import java.io.IOException;
 
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexSorter;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortFieldProvider;
 import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.NumericUtils;
 
 /** 
  * SortField for {@link SortedNumericDocValues}.
@@ -83,6 +89,86 @@ public class SortedNumericSortField extends SortField {
     this.type = type;
   }
 
+  /** A SortFieldProvider for this sort field */
+  public static final class Provider extends SortFieldProvider {
+
+    /** The name this provider is registered under */
+    public static final String NAME = "SortedNumericSortField";
+
+    /** Creates a new Provider */
+    public Provider() {
+      super(NAME);
+    }
+
+    @Override
+    public SortField readSortField(DataInput in) throws IOException {
+      SortedNumericSortField sf = new SortedNumericSortField(in.readString(), readType(in), in.readInt() == 1, readSelectorType(in));
+      if (in.readInt() == 1) {
+        switch (sf.type) {
+          case INT:
+            sf.setMissingValue(in.readInt());
+            break;
+          case LONG:
+            sf.setMissingValue(in.readLong());
+            break;
+          case FLOAT:
+            sf.setMissingValue(NumericUtils.sortableIntToFloat(in.readInt()));
+            break;
+          case DOUBLE:
+            sf.setMissingValue(NumericUtils.sortableLongToDouble(in.readLong()));
+            break;
+          default:
+            throw new AssertionError();
+        }
+      }
+      return sf;
+    }
+
+    @Override
+    public void writeSortField(SortField sf, DataOutput out) throws IOException {
+      assert sf instanceof SortedNumericSortField;
+      ((SortedNumericSortField)sf).serialize(out);
+    }
+  }
+
+  private static SortedNumericSelector.Type readSelectorType(DataInput in) throws IOException {
+    int selectorType = in.readInt();
+    if (selectorType >= SortedNumericSelector.Type.values().length) {
+      throw new IllegalArgumentException("Can't deserialize SortedNumericSortField - unknown selector type " + selectorType);
+    }
+    return SortedNumericSelector.Type.values()[selectorType];
+  }
+
+  private void serialize(DataOutput out) throws IOException {
+    out.writeString(getField());
+    out.writeString(type.toString());
+    out.writeInt(reverse ? 1 : 0);
+    out.writeInt(selector.ordinal());
+    if (missingValue == null) {
+      out.writeInt(0);
+    }
+    else {
+      out.writeInt(1);
+      // oh for switch expressions...
+      switch (type) {
+        case INT:
+          out.writeInt((int)missingValue);
+          break;
+        case LONG:
+          out.writeLong((long)missingValue);
+          break;
+        case FLOAT:
+          out.writeInt(NumericUtils.floatToSortableInt((float)missingValue));
+          break;
+        case DOUBLE:
+          out.writeLong(NumericUtils.doubleToSortableLong((double)missingValue));
+          break;
+        default:
+          throw new AssertionError();
+      }
+    }
+  }
+
   /** Returns the numeric type in use for this sort */
   public SortField.Type getNumericType() {
     return type;
@@ -170,4 +256,24 @@ public class SortedNumericSortField extends SortField {
         throw new AssertionError();
     }
   }
+
+  private NumericDocValues getValue(LeafReader reader) throws IOException {
+    return SortedNumericSelector.wrap(DocValues.getSortedNumeric(reader, getField()), selector, type);
+  }
+
+  @Override
+  public IndexSorter getIndexSorter() {
+    switch(type) {
+      case INT:
+        return new IndexSorter.IntSorter(Provider.NAME, (Integer)missingValue, reverse, this::getValue);
+      case LONG:
+        return new IndexSorter.LongSorter(Provider.NAME, (Long)missingValue, reverse, this::getValue);
+      case DOUBLE:
+        return new IndexSorter.DoubleSorter(Provider.NAME, (Double)missingValue, reverse, this::getValue);
+      case FLOAT:
+        return new IndexSorter.FloatSorter(Provider.NAME, (Float)missingValue, reverse, this::getValue);
+      default:
+        throw new AssertionError();
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java b/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
index b095c6e..2321a66 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
@@ -16,13 +16,17 @@
  */
 package org.apache.lucene.search;
 
-
 import java.io.IOException;
 
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexSorter;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortFieldProvider;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 
 /** 
  * SortField for {@link SortedSetDocValues}.
@@ -68,6 +72,60 @@ public class SortedSetSortField extends SortField {
     }
     this.selector = selector;
   }
+
+  /** A SortFieldProvider for this sort */
+  public static final class Provider extends SortFieldProvider {
+
+    /** The name this provider is registered under */
+    public static final String NAME = "SortedSetSortField";
+
+    /** Creates a new Provider */
+    public Provider() {
+      super(NAME);
+    }
+
+    @Override
+    public SortField readSortField(DataInput in) throws IOException {
+      SortField sf = new SortedSetSortField(in.readString(), in.readInt() == 1, readSelectorType(in));
+      int missingValue = in.readInt();
+      if (missingValue == 1) {
+        sf.setMissingValue(SortField.STRING_FIRST);
+      }
+      else if (missingValue == 2) {
+        sf.setMissingValue(SortField.STRING_LAST);
+      }
+      return sf;
+    }
+
+    @Override
+    public void writeSortField(SortField sf, DataOutput out) throws IOException {
+      assert sf instanceof SortedSetSortField;
+      ((SortedSetSortField)sf).serialize(out);
+    }
+  }
+
+  private static SortedSetSelector.Type readSelectorType(DataInput in) throws IOException {
+    int type = in.readInt();
+    if (type >= SortedSetSelector.Type.values().length) {
+      throw new IllegalArgumentException("Cannot deserialize SortedSetSortField: unknown selector type " + type);
+    }
+    return SortedSetSelector.Type.values()[type];
+  }
+
+  private void serialize(DataOutput out) throws IOException {
+    out.writeString(getField());
+    out.writeInt(reverse ? 1 : 0);
+    out.writeInt(selector.ordinal());
+    if (missingValue == SortField.STRING_FIRST) {
+      out.writeInt(1);
+    }
+    else if (missingValue == SortField.STRING_LAST) {
+      out.writeInt(2);
+    }
+    else {
+      out.writeInt(0);
+    }
+  }
   
   /** Returns the selector in use for this sort */
   public SortedSetSelector.Type getSelector() {
@@ -126,4 +184,13 @@ public class SortedSetSortField extends SortField {
       }
     };
   }
+
+  private SortedDocValues getValues(LeafReader reader) throws IOException {
+    return SortedSetSelector.wrap(DocValues.getSortedSet(reader, getField()), selector);
+  }
+
+  @Override
+  public IndexSorter getIndexSorter() {
+    return new IndexSorter.StringSorter(Provider.NAME, missingValue, reverse, this::getValues);
+  }
 }
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 611e237..2897a8a 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.lucene84.Lucene84Codec
+org.apache.lucene.codecs.lucene86.Lucene86Codec
diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.index.SortFieldProvider b/lucene/core/src/resources/META-INF/services/org.apache.lucene.index.SortFieldProvider
new file mode 100644
index 0000000..a96a47b
--- /dev/null
+++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.index.SortFieldProvider
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+
+org.apache.lucene.search.SortField$Provider
+org.apache.lucene.search.SortedNumericSortField$Provider
+org.apache.lucene.search.SortedSetSortField$Provider
\ No newline at end of file
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 6a3ce93..cccee73 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
@@ -17,9 +17,10 @@
 package org.apache.lucene.codecs.lucene50;
 
 
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene84.Lucene84Codec;
+import org.apache.lucene.codecs.lucene86.Lucene86Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@@ -28,12 +29,10 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.store.Directory;
 
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
-
 public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
   @Override
   protected Codec getCodec() {
-    return new Lucene84Codec(Mode.BEST_COMPRESSION);
+    return new Lucene86Codec(Mode.BEST_COMPRESSION);
   }
   
   /**
@@ -44,7 +43,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
     Directory dir = newDirectory();
     for (int i = 0; i < 10; i++) {
       IndexWriterConfig iwc = newIndexWriterConfig();
-      iwc.setCodec(new Lucene84Codec(RandomPicks.randomFrom(random(), Mode.values())));
+      iwc.setCodec(new Lucene86Codec(RandomPicks.randomFrom(random(), Mode.values())));
       IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
       Document doc = new Document();
       doc.add(new StoredField("field1", "value1"));
@@ -71,7 +70,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
   
   public void testInvalidOptions() {
     expectThrows(NullPointerException.class, () -> {
-      new Lucene84Codec(null);
+      new Lucene86Codec(null);
     });
 
     expectThrows(NullPointerException.class, () -> {
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java
index 4eadf05..b6e7268 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene80/TestLucene80NormsFormat.java
@@ -18,14 +18,14 @@ package org.apache.lucene.codecs.lucene80;
 
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene84.Lucene84Codec;
+import org.apache.lucene.codecs.lucene86.Lucene86Codec;
 import org.apache.lucene.index.BaseNormsFormatTestCase;
 
 /**
  * Tests Lucene80NormsFormat
  */
 public class TestLucene80NormsFormat extends BaseNormsFormatTestCase {
-  private final Codec codec = new Lucene84Codec();
+  private final Codec codec = new Lucene86Codec();
   
   @Override
   protected Codec getCodec() {
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86SegmentInfoFormat.java
similarity index 91%
rename from lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java
rename to lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86SegmentInfoFormat.java
index 3bf6a18..e462d3f 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene86/TestLucene86SegmentInfoFormat.java
@@ -14,14 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.codecs.lucene70;
+
+package org.apache.lucene.codecs.lucene86;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.BaseSegmentInfoFormatTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 
-public class TestLucene70SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
+public class TestLucene86SegmentInfoFormat extends BaseSegmentInfoFormatTestCase {
 
   @Override
   protected Version[] getVersions() {
@@ -32,4 +33,5 @@ public class TestLucene70SegmentInfoFormat extends BaseSegmentInfoFormatTestCase
   protected Codec getCodec() {
     return TestUtil.getDefaultCodec();
   }
+
 }
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 ae944de..042e2a8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -2146,7 +2146,7 @@ public class TestIndexSorting extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
       iwc.setIndexSort(Sort.RELEVANCE);
     });
-    assertEquals("invalid SortField type: must be one of [STRING, INT, FLOAT, LONG, DOUBLE] but got: <score>", expected.getMessage());
+    assertEquals("Cannot sort index with sort field <score>", expected.getMessage());
   }
 
   // you can't change the index sort on an existing index:
@@ -2498,6 +2498,7 @@ public class TestIndexSorting extends LuceneTestCase {
         System.out.println("  float=" + docValues.floatValue);
         System.out.println("  double=" + docValues.doubleValue);
         System.out.println("  bytes=" + new BytesRef(docValues.bytesValue));
+        System.out.println("  mvf=" + Arrays.toString(docValues.floatValues));
       }
 
       Document doc = new Document();
@@ -2741,7 +2742,7 @@ public class TestIndexSorting extends LuceneTestCase {
         Document doc = new Document();
         doc.add(dvs.get(j));
         IllegalArgumentException exc = expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc));
-        assertThat(exc.getMessage(), containsString("invalid doc value type"));
+        assertThat(exc.getMessage(), containsString("expected field [field] to be "));
         doc.clear();
         doc.add(dvs.get(i));
         w.addDocument(doc);
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 ee778ed..d982953 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -396,7 +396,7 @@ public class TestPointValues extends LuceneTestCase {
   public void testDifferentCodecs1() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(Codec.forName("Lucene84"));
+    iwc.setCodec(Codec.forName("Lucene86"));
     IndexWriter w = new IndexWriter(dir, iwc);
     Document doc = new Document();
     doc.add(new IntPoint("int", 1));
@@ -427,7 +427,7 @@ public class TestPointValues extends LuceneTestCase {
     w.close();
     
     iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    iwc.setCodec(Codec.forName("Lucene84"));
+    iwc.setCodec(Codec.forName("Lucene86"));
     w = new IndexWriter(dir, iwc);
     doc = new Document();
     doc.add(new IntPoint("int", 1));
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 2ccfd9a..3400f0e 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("Lucene84"));
+    iwc.setCodec(Codec.forName("Lucene86"));
     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("Lucene84"));
+    iwc.setCodec(Codec.forName("Lucene86"));
     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("Lucene84"));
+      iwc.setCodec(Codec.forName("Lucene86"));
       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("Lucene84"));
+    iwc.setCodec(Codec.forName("Lucene86"));
     RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
 
     doc = new Document();
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java
index b2d5b03..a14204c 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestFloatPointNearestNeighbor.java
@@ -243,7 +243,7 @@ public class TestFloatPointNearestNeighbor extends LuceneTestCase {
 
   private IndexWriterConfig getIndexWriterConfig() {
     IndexWriterConfig iwc = newIndexWriterConfig();
-    iwc.setCodec(Codec.forName("Lucene84"));
+    iwc.setCodec(Codec.forName("Lucene86"));
     return iwc;
   }
 }
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java
index 3ea2a4e..a149ace 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestNearest.java
@@ -246,7 +246,7 @@ public class TestNearest extends LuceneTestCase {
 
   private IndexWriterConfig getIndexWriterConfig() {
     IndexWriterConfig iwc = newIndexWriterConfig();
-    iwc.setCodec(Codec.forName("Lucene84"));
+    iwc.setCodec(Codec.forName("Lucene86"));
     return iwc;
   }
 }
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 9d80476..12c8902 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
@@ -16,9 +16,6 @@
  */
 package org.apache.lucene.search.suggest.document;
 
-import static org.apache.lucene.analysis.BaseTokenStreamTestCase.assertTokenStreamContents;
-import static org.hamcrest.core.IsEqual.equalTo;
-
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -32,6 +29,7 @@ import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CyclicBarrier;
 
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.TokenFilter;
@@ -41,7 +39,7 @@ import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene84.Lucene84Codec;
+import org.apache.lucene.codecs.lucene86.Lucene86Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.IntPoint;
@@ -69,7 +67,8 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import static org.apache.lucene.analysis.BaseTokenStreamTestCase.assertTokenStreamContents;
+import static org.hamcrest.core.IsEqual.equalTo;
 
 public class TestSuggestField extends LuceneTestCase {
 
@@ -888,7 +887,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 Lucene84Codec() {
+    Codec filterCodec = new Lucene86Codec() {
       CompletionPostingsFormat.FSTLoadMode fstLoadMode =
           RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values());
       PostingsFormat postingsFormat = new Completion84PostingsFormat(fstLoadMode);
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 e67e2a7..135ff38 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
@@ -1276,7 +1276,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("Lucene84", TestUtil.getDefaultCodec()) {
+    iwc.setCodec(new FilterCodec("Lucene86", TestUtil.getDefaultCodec()) {
       @Override
       public PointsFormat pointsFormat() {
         return new PointsFormat() {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
index a597ca5..eb7be93 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
@@ -1190,7 +1190,7 @@ public abstract class BaseXYPointTestCase extends LuceneTestCase {
     // Else seeds may not reproduce:
     iwc.setMergeScheduler(new SerialMergeScheduler());
     int pointsInLeaf = 2 + random().nextInt(4);
-    iwc.setCodec(new FilterCodec("Lucene84", TestUtil.getDefaultCodec()) {
+    iwc.setCodec(new FilterCodec("Lucene86", TestUtil.getDefaultCodec()) {
       @Override
       public PointsFormat pointsFormat() {
         return new PointsFormat() {
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 bd59e8c..aef11ac 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
@@ -16,17 +16,6 @@
  */
 package org.apache.lucene.util;
 
-import static org.apache.lucene.util.LuceneTestCase.INFOSTREAM;
-import static org.apache.lucene.util.LuceneTestCase.TEST_CODEC;
-import static org.apache.lucene.util.LuceneTestCase.TEST_DOCVALUESFORMAT;
-import static org.apache.lucene.util.LuceneTestCase.TEST_POSTINGSFORMAT;
-import static org.apache.lucene.util.LuceneTestCase.VERBOSE;
-import static org.apache.lucene.util.LuceneTestCase.assumeFalse;
-import static org.apache.lucene.util.LuceneTestCase.localeForLanguageTag;
-import static org.apache.lucene.util.LuceneTestCase.random;
-import static org.apache.lucene.util.LuceneTestCase.randomLocale;
-import static org.apache.lucene.util.LuceneTestCase.randomTimeZone;
-
 import java.io.PrintStream;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -34,6 +23,8 @@ import java.util.Locale;
 import java.util.Random;
 import java.util.TimeZone;
 
+import com.carrotsearch.randomizedtesting.RandomizedContext;
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -43,7 +34,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.lucene84.Lucene84Codec;
+import org.apache.lucene.codecs.lucene86.Lucene86Codec;
 import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
@@ -54,8 +45,16 @@ import org.apache.lucene.util.LuceneTestCase.LiveIWCFlushMode;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.junit.internal.AssumptionViolatedException;
 
-import com.carrotsearch.randomizedtesting.RandomizedContext;
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import static org.apache.lucene.util.LuceneTestCase.INFOSTREAM;
+import static org.apache.lucene.util.LuceneTestCase.TEST_CODEC;
+import static org.apache.lucene.util.LuceneTestCase.TEST_DOCVALUESFORMAT;
+import static org.apache.lucene.util.LuceneTestCase.TEST_POSTINGSFORMAT;
+import static org.apache.lucene.util.LuceneTestCase.VERBOSE;
+import static org.apache.lucene.util.LuceneTestCase.assumeFalse;
+import static org.apache.lucene.util.LuceneTestCase.localeForLanguageTag;
+import static org.apache.lucene.util.LuceneTestCase.random;
+import static org.apache.lucene.util.LuceneTestCase.randomLocale;
+import static org.apache.lucene.util.LuceneTestCase.randomTimeZone;
 
 /**
  * Setup and restore suite-level environment (fine grained junk that 
@@ -189,7 +188,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
     } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
       codec = CompressingCodec.randomInstance(random);
     } else if ("Lucene84".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene84"))) {
-      codec = new Lucene84Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values())
+      codec = new Lucene86Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values())
       );
     } else if (!"random".equals(TEST_CODEC)) {
       codec = Codec.forName(TEST_CODEC);
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 f0697f9..2dc9ead 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.lucene80.Lucene80DocValuesFormat;
-import org.apache.lucene.codecs.lucene84.Lucene84Codec;
 import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
+import org.apache.lucene.codecs.lucene86.Lucene86Codec;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
@@ -919,7 +919,7 @@ public final class TestUtil {
    * This may be different than {@link Codec#getDefault()} because that is randomized. 
    */
   public static Codec getDefaultCodec() {
-    return new Lucene84Codec();
+    return new Lucene86Codec();
   }
   
   /** 
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 fa34edd..8a591c8 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.lucene84.Lucene84Codec;
+import org.apache.lucene.codecs.lucene86.Lucene86Codec;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 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 Lucene84Codec(compressionMode) {
+    codec = new Lucene86Codec(compressionMode) {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);