You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2020/09/03 11:05:32 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9484: Allow sorting an index after the fact (#1789)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new a7bdc68  LUCENE-9484: Allow sorting an index after the fact (#1789)
a7bdc68 is described below

commit a7bdc6893e21954ed9f6d8bce256a4a9c917310b
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Thu Sep 3 12:54:40 2020 +0200

    LUCENE-9484: Allow sorting an index after the fact (#1789)
    
    Today we need to decide on an index sorting before we create the index.
    In some situations it might make a lot of sense to sort an index afterwards
    when the index doesn't change anymore or to compress older indices.
    This change adds the ability to wrap readers from an unsorted index and merge it
    into a sorted index by using IW#addIndices.
---
 lucene/CHANGES.txt                                 |    4 +
 .../apache/lucene/index/BinaryDocValuesWriter.java |   68 +-
 .../apache/lucene/index/FreqProxTermsWriter.java   |  521 +++++++-
 .../org/apache/lucene/index/NormValuesWriter.java  |    4 +-
 .../lucene/index/NumericDocValuesWriter.java       |   72 +-
 .../apache/lucene/index/SortedDocValuesWriter.java |   70 +-
 .../lucene/index/SortedNumericDocValuesWriter.java |   70 +-
 .../lucene/index/SortedSetDocValuesWriter.java     |   77 +-
 .../apache/lucene/index/SortingCodecReader.java    |  513 ++++++++
 .../org/apache/lucene/index/SortingLeafReader.java | 1268 --------------------
 .../lucene/index/TestSortingCodecReader.java       |  224 ++++
 11 files changed, 1604 insertions(+), 1287 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 92f94fc..2a6a3ae 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -21,6 +21,10 @@ New Features
   small segments on getReader, subject to a configurable timeout, to improve
   search performance by reducing the number of small segments for searching. (Simon Willnauer)
 
+* LUCENE-9484: Allow sorting an index after it was created. With SortingCodecReader, existing
+  unsorted segments can be wrapped and merged into a fresh index using IndexWriter#addIndices
+  API. (Simon Willnauer, Adrien Grand)
+
 Improvements
 ---------------------
 
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 e213a48..775f325 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
@@ -24,6 +24,7 @@ import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.Counter;
@@ -99,7 +100,7 @@ class BinaryDocValuesWriter extends DocValuesWriter<BinaryDocValues> {
     bytesUsed = newBytesUsed;
   }
 
-  private SortingLeafReader.CachedBinaryDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, BinaryDocValues oldValues) throws IOException {
+  static CachedBinaryDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, BinaryDocValues oldValues) throws IOException {
     FixedBitSet docsWithField = new FixedBitSet(maxDoc);
     BytesRef[] values = new BytesRef[maxDoc];
     while (true) {
@@ -111,7 +112,7 @@ class BinaryDocValuesWriter extends DocValuesWriter<BinaryDocValues> {
       docsWithField.set(newDocID);
       values[newDocID] = BytesRef.deepCopyOf(oldValues.binaryValue());
     }
-    return new SortingLeafReader.CachedBinaryDVs(values, docsWithField);
+    return new CachedBinaryDVs(values, docsWithField);
   }
 
   @Override
@@ -128,7 +129,7 @@ class BinaryDocValuesWriter extends DocValuesWriter<BinaryDocValues> {
     if (finalLengths == null) {
       finalLengths = this.lengths.build();
     }
-    final SortingLeafReader.CachedBinaryDVs sorted;
+    final CachedBinaryDVs sorted;
     if (sortMap != null) {
       sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap,
           new BufferedBinaryDocValues(finalLengths, maxLength, bytes.getDataInput(), docsWithField.iterator()));
@@ -145,7 +146,7 @@ class BinaryDocValuesWriter extends DocValuesWriter<BinaryDocValues> {
                                   if (sorted == null) {
                                     return new BufferedBinaryDocValues(finalLengths, maxLength, bytes.getDataInput(), docsWithField.iterator());
                                   } else {
-                                    return new SortingLeafReader.SortingBinaryDocValues(sorted);
+                                    return new SortingBinaryDocValues(sorted);
                                   }
                                 }
                               });
@@ -202,4 +203,63 @@ class BinaryDocValuesWriter extends DocValuesWriter<BinaryDocValues> {
       return value.get();
     }
   }
+
+  static class SortingBinaryDocValues extends BinaryDocValues {
+    private final CachedBinaryDVs dvs;
+    private int docID = -1;
+    private long cost = -1;
+
+    SortingBinaryDocValues(CachedBinaryDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+      return docID;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      return dvs.values[docID];
+    }
+
+    @Override
+    public long cost() {
+      if (cost == -1) {
+        cost = dvs.docsWithField.cardinality();
+      }
+      return cost;
+    }
+  }
+
+  static class CachedBinaryDVs {
+    // TODO: at least cutover to BytesRefArray here:
+    private final BytesRef[] values;
+    private final BitSet docsWithField;
+
+    CachedBinaryDVs(BytesRef[] values, BitSet docsWithField) {
+      this.values = values;
+      this.docsWithField = docsWithField;
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index bbc7b18..702218b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -26,9 +26,16 @@ import java.util.Map;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.TimSorter;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 final class FreqProxTermsWriter extends TermsHash {
 
@@ -88,7 +95,20 @@ final class FreqProxTermsWriter extends TermsHash {
     Fields fields = new FreqProxFields(allFields);
     applyDeletes(state, fields);
     if (sortMap != null) {
-      fields = new SortingLeafReader.SortingFields(fields, state.fieldInfos, sortMap);
+      final Sorter.DocMap docMap = sortMap;
+      final FieldInfos infos = state.fieldInfos;
+      fields = new FilterLeafReader.FilterFields(fields) {
+
+        @Override
+        public Terms terms(final String field) throws IOException {
+          Terms terms = in.terms(field);
+          if (terms == null) {
+            return null;
+          } else {
+            return new SortingTerms(terms, infos.fieldInfo(field).getIndexOptions(), docMap);
+          }
+        }
+      };
     }
 
     FieldsConsumer consumer = state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state);
@@ -110,4 +130,503 @@ final class FreqProxTermsWriter extends TermsHash {
   public TermsHashPerField addField(FieldInvertState invertState, FieldInfo fieldInfo) {
     return new FreqProxTermsWriterPerField(invertState, this, fieldInfo, nextTermsHash.addField(invertState, fieldInfo));
   }
+
+  static class SortingTerms extends FilterLeafReader.FilterTerms {
+
+    private final Sorter.DocMap docMap;
+    private final IndexOptions indexOptions;
+
+    SortingTerms(final Terms in, IndexOptions indexOptions, final Sorter.DocMap docMap) {
+      super(in);
+      this.docMap = docMap;
+      this.indexOptions = indexOptions;
+    }
+
+    @Override
+    public TermsEnum iterator() throws IOException {
+      return new SortingTermsEnum(in.iterator(), docMap, indexOptions, hasPositions());
+    }
+
+    @Override
+    public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm)
+        throws IOException {
+      return new SortingTermsEnum(in.intersect(compiled, startTerm), docMap, indexOptions, hasPositions());
+    }
+
+  }
+
+  private static class SortingTermsEnum extends FilterLeafReader.FilterTermsEnum {
+
+    final Sorter.DocMap docMap; // pkg-protected to avoid synthetic accessor methods
+    private final IndexOptions indexOptions;
+    private final boolean hasPositions;
+
+    SortingTermsEnum(final TermsEnum in, Sorter.DocMap docMap, IndexOptions indexOptions, boolean hasPositions) {
+      super(in);
+      this.docMap = docMap;
+      this.indexOptions = indexOptions;
+      this.hasPositions = hasPositions;
+    }
+
+    @Override
+    public PostingsEnum postings( PostingsEnum reuse, final int flags) throws IOException {
+
+      if (hasPositions && PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS)) {
+        final PostingsEnum inReuse;
+        final SortingPostingsEnum wrapReuse;
+        if (reuse != null && reuse instanceof SortingPostingsEnum) {
+          // if we're asked to reuse the given DocsEnum and it is Sorting, return
+          // the wrapped one, since some Codecs expect it.
+          wrapReuse = (SortingPostingsEnum) reuse;
+          inReuse = wrapReuse.getWrapped();
+        } else {
+          wrapReuse = null;
+          inReuse = reuse;
+        }
+
+        final PostingsEnum inDocsAndPositions = in.postings(inReuse, flags);
+        // we ignore the fact that offsets may be stored but not asked for,
+        // since this code is expected to be used during addIndexes which will
+        // ask for everything. if that assumption changes in the future, we can
+        // factor in whether 'flags' says offsets are not required.
+        final boolean storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+        return new SortingPostingsEnum(docMap.size(), wrapReuse, inDocsAndPositions, docMap, storeOffsets);
+      }
+
+      final PostingsEnum inReuse;
+      final SortingDocsEnum wrapReuse;
+      if (reuse != null && reuse instanceof SortingDocsEnum) {
+        // if we're asked to reuse the given DocsEnum and it is Sorting, return
+        // the wrapped one, since some Codecs expect it.
+        wrapReuse = (SortingDocsEnum) reuse;
+        inReuse = wrapReuse.getWrapped();
+      } else {
+        wrapReuse = null;
+        inReuse = reuse;
+      }
+
+      final PostingsEnum inDocs = in.postings(inReuse, flags);
+      final boolean withFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >=0 && PostingsEnum.featureRequested(flags, PostingsEnum.FREQS);
+      return new SortingDocsEnum(docMap.size(), wrapReuse, inDocs, withFreqs, docMap);
+    }
+
+  }
+
+  static class SortingDocsEnum extends FilterLeafReader.FilterPostingsEnum {
+
+    private static final class DocFreqSorter extends TimSorter {
+
+      private int[] docs;
+      private int[] freqs;
+      private final int[] tmpDocs;
+      private int[] tmpFreqs;
+
+      DocFreqSorter(int maxDoc) {
+        super(maxDoc / 64);
+        this.tmpDocs = new int[maxDoc / 64];
+      }
+
+      public void reset(int[] docs, int[] freqs) {
+        this.docs = docs;
+        this.freqs = freqs;
+        if (freqs != null && tmpFreqs == null) {
+          tmpFreqs = new int[tmpDocs.length];
+        }
+      }
+
+      @Override
+      protected int compare(int i, int j) {
+        return docs[i] - docs[j];
+      }
+
+      @Override
+      protected void swap(int i, int j) {
+        int tmpDoc = docs[i];
+        docs[i] = docs[j];
+        docs[j] = tmpDoc;
+
+        if (freqs != null) {
+          int tmpFreq = freqs[i];
+          freqs[i] = freqs[j];
+          freqs[j] = tmpFreq;
+        }
+      }
+
+      @Override
+      protected void copy(int src, int dest) {
+        docs[dest] = docs[src];
+        if (freqs != null) {
+          freqs[dest] = freqs[src];
+        }
+      }
+
+      @Override
+      protected void save(int i, int len) {
+        System.arraycopy(docs, i, tmpDocs, 0, len);
+        if (freqs != null) {
+          System.arraycopy(freqs, i, tmpFreqs, 0, len);
+        }
+      }
+
+      @Override
+      protected void restore(int i, int j) {
+        docs[j] = tmpDocs[i];
+        if (freqs != null) {
+          freqs[j] = tmpFreqs[i];
+        }
+      }
+
+      @Override
+      protected int compareSaved(int i, int j) {
+        return tmpDocs[i] - docs[j];
+      }
+    }
+
+    private final int maxDoc;
+    private final DocFreqSorter sorter;
+    private int[] docs;
+    private int[] freqs;
+    private int docIt = -1;
+    private final int upto;
+    private final boolean withFreqs;
+
+    SortingDocsEnum(int maxDoc, SortingDocsEnum reuse, final PostingsEnum in, boolean withFreqs, final Sorter.DocMap docMap) throws IOException {
+      super(in);
+      this.maxDoc = maxDoc;
+      this.withFreqs = withFreqs;
+      if (reuse != null) {
+        if (reuse.maxDoc == maxDoc) {
+          sorter = reuse.sorter;
+        } else {
+          sorter = new DocFreqSorter(maxDoc);
+        }
+        docs = reuse.docs;
+        freqs = reuse.freqs; // maybe null
+      } else {
+        docs = new int[64];
+        sorter = new DocFreqSorter(maxDoc);
+      }
+      docIt = -1;
+      int i = 0;
+      int doc;
+      if (withFreqs) {
+        if (freqs == null || freqs.length < docs.length) {
+          freqs = new int[docs.length];
+        }
+        while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS){
+          if (i >= docs.length) {
+            docs = ArrayUtil.grow(docs, docs.length + 1);
+            freqs = ArrayUtil.grow(freqs, freqs.length + 1);
+          }
+          docs[i] = docMap.oldToNew(doc);
+          freqs[i] = in.freq();
+          ++i;
+        }
+      } else {
+        freqs = null;
+        while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS){
+          if (i >= docs.length) {
+            docs = ArrayUtil.grow(docs, docs.length + 1);
+          }
+          docs[i++] = docMap.oldToNew(doc);
+        }
+      }
+      // TimSort can save much time compared to other sorts in case of
+      // reverse sorting, or when sorting a concatenation of sorted readers
+      sorter.reset(docs, freqs);
+      sorter.sort(0, i);
+      upto = i;
+    }
+
+    // for testing
+    boolean reused(PostingsEnum other) {
+      if (other == null || !(other instanceof SortingDocsEnum)) {
+        return false;
+      }
+      return docs == ((SortingDocsEnum) other).docs;
+    }
+
+    @Override
+    public int advance(final int target) throws IOException {
+      // need to support it for checkIndex, but in practice it won't be called, so
+      // don't bother to implement efficiently for now.
+      return slowAdvance(target);
+    }
+
+    @Override
+    public int docID() {
+      return docIt < 0 ? -1 : docIt >= upto ? NO_MORE_DOCS : docs[docIt];
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return withFreqs && docIt < upto ? freqs[docIt] : 1;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (++docIt >= upto) return NO_MORE_DOCS;
+      return docs[docIt];
+    }
+
+    /** Returns the wrapped {@link PostingsEnum}. */
+    PostingsEnum getWrapped() {
+      return in;
+    }
+
+    // we buffer up docs/freqs only, don't forward any positions requests to underlying enum
+
+    @Override
+    public int nextPosition() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return null;
+    }
+  }
+
+  static class SortingPostingsEnum extends FilterLeafReader.FilterPostingsEnum {
+
+    /**
+     * A {@link TimSorter} which sorts two parallel arrays of doc IDs and
+     * offsets in one go. Everyti
+     * me a doc ID is 'swapped', its corresponding offset
+     * is swapped too.
+     */
+    private static final class DocOffsetSorter extends TimSorter {
+
+      private int[] docs;
+      private long[] offsets;
+      private final int[] tmpDocs;
+      private final long[] tmpOffsets;
+
+      public DocOffsetSorter(int maxDoc) {
+        super(maxDoc / 64);
+        this.tmpDocs = new int[maxDoc / 64];
+        this.tmpOffsets = new long[maxDoc / 64];
+      }
+
+      public void reset(int[] docs, long[] offsets) {
+        this.docs = docs;
+        this.offsets = offsets;
+      }
+
+      @Override
+      protected int compare(int i, int j) {
+        return docs[i] - docs[j];
+      }
+
+      @Override
+      protected void swap(int i, int j) {
+        int tmpDoc = docs[i];
+        docs[i] = docs[j];
+        docs[j] = tmpDoc;
+
+        long tmpOffset = offsets[i];
+        offsets[i] = offsets[j];
+        offsets[j] = tmpOffset;
+      }
+
+      @Override
+      protected void copy(int src, int dest) {
+        docs[dest] = docs[src];
+        offsets[dest] = offsets[src];
+      }
+
+      @Override
+      protected void save(int i, int len) {
+        System.arraycopy(docs, i, tmpDocs, 0, len);
+        System.arraycopy(offsets, i, tmpOffsets, 0, len);
+      }
+
+      @Override
+      protected void restore(int i, int j) {
+        docs[j] = tmpDocs[i];
+        offsets[j] = tmpOffsets[i];
+      }
+
+      @Override
+      protected int compareSaved(int i, int j) {
+        return tmpDocs[i] - docs[j];
+      }
+    }
+
+    private final int maxDoc;
+    private final DocOffsetSorter sorter;
+    private int[] docs;
+    private long[] offsets;
+    private final int upto;
+
+    private final ByteBuffersDataInput postingInput;
+    private final boolean storeOffsets;
+
+    private int docIt = -1;
+    private int pos;
+    private int startOffset = -1;
+    private int endOffset = -1;
+    private final BytesRef payload;
+    private int currFreq;
+
+    private final ByteBuffersDataOutput buffer;
+
+    SortingPostingsEnum(int maxDoc, SortingPostingsEnum reuse, final PostingsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
+      super(in);
+      this.maxDoc = maxDoc;
+      this.storeOffsets = storeOffsets;
+      if (reuse != null) {
+        docs = reuse.docs;
+        offsets = reuse.offsets;
+        payload = reuse.payload;
+        buffer = reuse.buffer;
+        buffer.reset();
+        if (reuse.maxDoc == maxDoc) {
+          sorter = reuse.sorter;
+        } else {
+          sorter = new DocOffsetSorter(maxDoc);
+        }
+      } else {
+        docs = new int[32];
+        offsets = new long[32];
+        payload = new BytesRef(32);
+        buffer = ByteBuffersDataOutput.newResettableInstance();
+        sorter = new DocOffsetSorter(maxDoc);
+      }
+
+      int doc;
+      int i = 0;
+      while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+        if (i == docs.length) {
+          final int newLength = ArrayUtil.oversize(i + 1, 4);
+          docs = ArrayUtil.growExact(docs, newLength);
+          offsets = ArrayUtil.growExact(offsets, newLength);
+        }
+        docs[i] = docMap.oldToNew(doc);
+        offsets[i] = buffer.size();
+        addPositions(in, buffer);
+        i++;
+      }
+      upto = i;
+      sorter.reset(docs, offsets);
+      sorter.sort(0, upto);
+
+      this.postingInput = buffer.toDataInput();
+    }
+
+    // for testing
+    boolean reused(PostingsEnum other) {
+      if (other == null || !(other instanceof SortingPostingsEnum)) {
+        return false;
+      }
+      return docs == ((SortingPostingsEnum) other).docs;
+    }
+
+    private void addPositions(final PostingsEnum in, final DataOutput out) throws IOException {
+      int freq = in.freq();
+      out.writeVInt(freq);
+      int previousPosition = 0;
+      int previousEndOffset = 0;
+      for (int i = 0; i < freq; i++) {
+        final int pos = in.nextPosition();
+        final BytesRef payload = in.getPayload();
+        // The low-order bit of token is set only if there is a payload, the
+        // previous bits are the delta-encoded position.
+        final int token = (pos - previousPosition) << 1 | (payload == null ? 0 : 1);
+        out.writeVInt(token);
+        previousPosition = pos;
+        if (storeOffsets) { // don't encode offsets if they are not stored
+          final int startOffset = in.startOffset();
+          final int endOffset = in.endOffset();
+          out.writeVInt(startOffset - previousEndOffset);
+          out.writeVInt(endOffset - startOffset);
+          previousEndOffset = endOffset;
+        }
+        if (payload != null) {
+          out.writeVInt(payload.length);
+          out.writeBytes(payload.bytes, payload.offset, payload.length);
+        }
+      }
+    }
+
+    @Override
+    public int advance(final int target) throws IOException {
+      // need to support it for checkIndex, but in practice it won't be called, so
+      // don't bother to implement efficiently for now.
+      return slowAdvance(target);
+    }
+
+    @Override
+    public int docID() {
+      return docIt < 0 ? -1 : docIt >= upto ? NO_MORE_DOCS : docs[docIt];
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return endOffset;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return currFreq;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return payload.length == 0 ? null : payload;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (++docIt >= upto) return DocIdSetIterator.NO_MORE_DOCS;
+      postingInput.seek(offsets[docIt]);
+      currFreq = postingInput.readVInt();
+      // reset variables used in nextPosition
+      pos = 0;
+      endOffset = 0;
+      return docs[docIt];
+    }
+
+    @Override
+    public int nextPosition() throws IOException {
+      final int token = postingInput.readVInt();
+      pos += token >>> 1;
+      if (storeOffsets) {
+        startOffset = endOffset + postingInput.readVInt();
+        endOffset = startOffset + postingInput.readVInt();
+      }
+      if ((token & 1) != 0) {
+        payload.offset = 0;
+        payload.length = postingInput.readVInt();
+        if (payload.length > payload.bytes.length) {
+          payload.bytes = new byte[ArrayUtil.oversize(payload.length, 1)];
+        }
+        postingInput.readBytes(payload.bytes, 0, payload.length);
+      } else {
+        payload.length = 0;
+      }
+      return pos;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return startOffset;
+    }
+
+    /** Returns the wrapped {@link PostingsEnum}. */
+    PostingsEnum getWrapped() {
+      return in;
+    }
+  }
+
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
index 0cfb081..2964352 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
@@ -70,7 +70,7 @@ class NormValuesWriter {
 
   public void flush(SegmentWriteState state, Sorter.DocMap sortMap, NormsConsumer normsConsumer) throws IOException {
     final PackedLongValues values = pending.build();
-    final SortingLeafReader.CachedNumericDVs sorted;
+    final NumericDocValuesWriter.CachedNumericDVs sorted;
     if (sortMap != null) {
       sorted = NumericDocValuesWriter.sortDocValues(state.segmentInfo.maxDoc(), sortMap,
           new BufferedNorms(values, docsWithField.iterator()));
@@ -87,7 +87,7 @@ class NormValuesWriter {
                                    if (sorted == null) {
                                      return new BufferedNorms(values, docsWithField.iterator());
                                    } else {
-                                     return new SortingLeafReader.SortingNumericDocValues(sorted);
+                                     return new NumericDocValuesWriter.SortingNumericDocValues(sorted);
                                    }
                                   }
 
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 87de98f..a696bca 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.packed.PackedInts;
@@ -40,7 +41,7 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
   private final FieldInfo fieldInfo;
   private int lastDocID = -1;
 
-  public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+  NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
     docsWithField = new DocsWithFieldSet();
     bytesUsed = pending.ramBytesUsed() + docsWithField.ramBytesUsed();
@@ -76,7 +77,7 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
     return new BufferedNumericDocValues(finalValues, docsWithField.iterator());
   }
 
-  static SortingLeafReader.CachedNumericDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues) throws IOException {
+  static CachedNumericDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues) throws IOException {
     FixedBitSet docsWithField = new FixedBitSet(maxDoc);
     long[] values = new long[maxDoc];
     while (true) {
@@ -88,7 +89,7 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
       docsWithField.set(newDocID);
       values[newDocID] = oldDocValues.longValue();
     }
-    return new SortingLeafReader.CachedNumericDVs(values, docsWithField);
+    return new CachedNumericDVs(values, docsWithField);
   }
 
   @Override
@@ -96,7 +97,7 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
     if (finalValues == null) {
       finalValues = pending.build();
     }
-    final SortingLeafReader.CachedNumericDVs sorted;
+    final CachedNumericDVs sorted;
     if (sortMap != null) {
       NumericDocValues oldValues = new BufferedNumericDocValues(finalValues, docsWithField.iterator());
       sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap, oldValues);
@@ -114,7 +115,7 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
                                    if (sorted == null) {
                                      return new BufferedNumericDocValues(finalValues, docsWithField.iterator());
                                    } else {
-                                     return new SortingLeafReader.SortingNumericDocValues(sorted);
+                                     return new SortingNumericDocValues(sorted);
                                    }
                                  }
                                });
@@ -165,4 +166,65 @@ class NumericDocValuesWriter extends DocValuesWriter<NumericDocValues> {
       return value;
     }
   }
+
+  static class SortingNumericDocValues extends NumericDocValues {
+
+    private final CachedNumericDVs dvs;
+    private int docID = -1;
+    private long cost = -1;
+
+    SortingNumericDocValues(CachedNumericDVs dvs) {
+      this.dvs = dvs;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      if (docID+1 == dvs.docsWithField.length()) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = dvs.docsWithField.nextSetBit(docID+1);
+      }
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc() instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      // needed in IndexSorter#{Long|Int|Double|Float}Sorter
+      docID = target;
+      return dvs.docsWithField.get(target);
+    }
+
+    @Override
+    public long longValue() {
+      return dvs.values[docID];
+    }
+
+    @Override
+    public long cost() {
+      if (cost == -1) {
+        cost = dvs.docsWithField.cardinality();
+      }
+      return cost;
+    }
+  }
+
+  static class CachedNumericDVs {
+    private final long[] values;
+    private final BitSet docsWithField;
+
+    CachedNumericDVs(long[] values, BitSet docsWithField) {
+      this.values = values;
+      this.docsWithField = docsWithField;
+    }
+  }
 }
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 2252f00..4ed81f7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
@@ -160,7 +160,7 @@ class SortedDocValuesWriter extends DocValuesWriter<SortedDocValues> {
                                   if (sorted == null) {
                                    return buf;
                                   }
-                                  return new SortingLeafReader.SortingSortedDocValues(buf, sorted);
+                                  return new SortingSortedDocValues(buf, sorted);
                                 }
                               });
   }
@@ -233,4 +233,72 @@ class SortedDocValuesWriter extends DocValuesWriter<SortedDocValues> {
     }
   }
 
+  static class SortingSortedDocValues extends SortedDocValues {
+
+    private final SortedDocValues in;
+    private final int[] ords;
+    private int docID = -1;
+
+    SortingSortedDocValues(SortedDocValues in, int[] ords) {
+      this.in = in;
+      this.ords = ords;
+      assert ords != null;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      while (true) {
+        docID++;
+        if (docID == ords.length) {
+          docID = NO_MORE_DOCS;
+          break;
+        }
+        if (ords[docID] != -1) {
+          break;
+        }
+        // skip missing docs
+      }
+
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      // needed in IndexSorter#StringSorter
+      docID = target;
+      return ords[target] != -1;
+    }
+
+    @Override
+    public int ordValue() {
+      return ords[docID];
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+
+    @Override
+    public BytesRef lookupOrd(int ord) throws IOException {
+      return in.lookupOrd(ord);
+    }
+
+    @Override
+    public int getValueCount() {
+      return in.getValueCount();
+    }
+  }
+
 }
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 83c394f..0acf93b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java
@@ -108,7 +108,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter<SortedNumericDocValue
     return new BufferedSortedNumericDocValues(finalValues, finalValuesCount, docsWithField.iterator());
   }
 
-  private long[][] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedNumericDocValues oldValues) throws IOException {
+  static long[][] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedNumericDocValues oldValues) throws IOException {
     long[][] values = new long[maxDoc][];
     int docID;
     while ((docID = oldValues.nextDoc()) != NO_MORE_DOCS) {
@@ -155,7 +155,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter<SortedNumericDocValue
                                          if (sorted == null) {
                                            return buf;
                                          } else {
-                                           return new SortingLeafReader.SortingSortedNumericDocValues(buf, sorted);
+                                           return new SortingSortedNumericDocValues(buf, sorted);
                                          }
                                        }
                                      });
@@ -168,7 +168,7 @@ class SortedNumericDocValuesWriter extends DocValuesWriter<SortedNumericDocValue
     private int valueCount;
     private int valueUpto;
 
-    public BufferedSortedNumericDocValues(PackedLongValues values, PackedLongValues valueCounts, DocIdSetIterator docsWithField) {
+    BufferedSortedNumericDocValues(PackedLongValues values, PackedLongValues valueCounts, DocIdSetIterator docsWithField) {
       valuesIter = values.iterator();
       valueCountsIter = valueCounts.iterator();
       this.docsWithField = docsWithField;
@@ -223,4 +223,68 @@ class SortedNumericDocValuesWriter extends DocValuesWriter<SortedNumericDocValue
     }
   }
 
+  static class SortingSortedNumericDocValues extends SortedNumericDocValues {
+    private final SortedNumericDocValues in;
+    private final long[][] values;
+    private int docID = -1;
+    private int upto;
+
+    SortingSortedNumericDocValues(SortedNumericDocValues in, long[][] values) {
+      this.in = in;
+      this.values = values;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      while (true) {
+        docID++;
+        if (docID == values.length) {
+          docID = NO_MORE_DOCS;
+          break;
+        }
+        if (values[docID] != null) {
+          break;
+        }
+        // skip missing docs
+      }
+      upto = 0;
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      docID = target;
+      upto = 0;
+      return values[docID] != null;
+    }
+
+    @Override
+    public long nextValue() {
+      if (upto == values[docID].length) {
+        throw new AssertionError();
+      } else {
+        return values[docID][upto++];
+      }
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+
+    @Override
+    public int docValueCount() {
+      return values[docID].length;
+    }
+  }
 }
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 022b17d..c444edd 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
@@ -139,7 +139,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter<SortedSetDocValues> {
     bytesUsed = newBytesUsed;
   }
 
-  private long[][] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedSetDocValues oldValues) throws IOException {
+  static long[][] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedSetDocValues oldValues) throws IOException {
     long[][] ords = new long[maxDoc][];
     int docID;
     while ((docID = oldValues.nextDoc()) != NO_MORE_DOCS) {
@@ -222,7 +222,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter<SortedSetDocValues> {
                                      if (sorted == null) {
                                        return buf;
                                      } else {
-                                       return new SortingLeafReader.SortingSortedSetDocValues(buf, sorted);
+                                       return new SortingSortedSetDocValues(buf, sorted);
                                      }
                                    }
                                  });
@@ -241,7 +241,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter<SortedSetDocValues> {
     private int ordCount;
     private int ordUpto;
 
-    public BufferedSortedSetDocValues(int[] sortedValues, int[] ordMap, BytesRefHash hash, PackedLongValues ords, PackedLongValues ordCounts, int maxCount, DocIdSetIterator docsWithField) {
+    BufferedSortedSetDocValues(int[] sortedValues, int[] ordMap, BytesRefHash hash, PackedLongValues ords, PackedLongValues ordCounts, int maxCount, DocIdSetIterator docsWithField) {
       this.currentDoc = new int[maxCount];
       this.sortedValues = sortedValues;
       this.ordMap = ordMap;
@@ -308,4 +308,75 @@ class SortedSetDocValuesWriter extends DocValuesWriter<SortedSetDocValues> {
     }
   }
 
+  static class SortingSortedSetDocValues extends SortedSetDocValues {
+
+    private final SortedSetDocValues in;
+    private final long[][] ords;
+    private int docID = -1;
+    private int ordUpto;
+
+    SortingSortedSetDocValues(SortedSetDocValues in, long[][] ords) {
+      this.in = in;
+      this.ords = ords;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      while (true) {
+        docID++;
+        if (docID == ords.length) {
+          docID = NO_MORE_DOCS;
+          break;
+        }
+        if (ords[docID] != null) {
+          break;
+        }
+        // skip missing docs
+      }
+      ordUpto = 0;
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      throw new UnsupportedOperationException("use nextDoc instead");
+    }
+
+    @Override
+    public boolean advanceExact(int target) throws IOException {
+      // needed in IndexSorter#StringSorter
+      docID = target;
+      ordUpto = 0;
+      return ords[docID] != null;
+    }
+
+    @Override
+    public long nextOrd() {
+      if (ordUpto == ords[docID].length) {
+        return NO_MORE_ORDS;
+      } else {
+        return ords[docID][ordUpto++];
+      }
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+
+    @Override
+    public BytesRef lookupOrd(long ord) throws IOException {
+      return in.lookupOrd(ord);
+    }
+
+    @Override
+    public long getValueCount() {
+      return in.getValueCount();
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
new file mode 100644
index 0000000..cc7739c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
@@ -0,0 +1,513 @@
+/*
+ * 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.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/**
+ * An {@link org.apache.lucene.index.CodecReader} which supports sorting documents by a given
+ * {@link Sort}. This can be used to re-sort and index after it's been created by wrapping all
+ * readers of the index with this reader and adding it to a fresh IndexWriter via
+ * {@link IndexWriter#addIndexes(CodecReader...)}.
+ *
+ * @lucene.experimental
+ */
+public final class SortingCodecReader extends FilterCodecReader {
+
+  private final Map<String, NumericDocValuesWriter.CachedNumericDVs> cachedNumericDVs = new HashMap<>();
+
+  private final Map<String, BinaryDocValuesWriter.CachedBinaryDVs> cachedBinaryDVs = new HashMap<>();
+
+  private final Map<String, int[]> cachedSortedDVs = new HashMap<>();
+
+  // TODO: pack long[][] into an int[] (offset) and long[] instead:
+  private final Map<String, long[][]> cachedSortedSetDVs = new HashMap<>();
+
+  private final Map<String, long[][]> cachedSortedNumericDVs = new HashMap<>();
+
+  private static class SortingBits implements Bits {
+
+    private final Bits in;
+    private final Sorter.DocMap docMap;
+
+    SortingBits(final Bits in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public boolean get(int index) {
+      return in.get(docMap.newToOld(index));
+    }
+
+    @Override
+    public int length() {
+      return in.length();
+    }
+  }
+
+  private static class SortingPointValues extends PointValues {
+
+    private final PointValues in;
+    private final Sorter.DocMap docMap;
+
+    SortingPointValues(final PointValues in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public void intersect(IntersectVisitor visitor) throws IOException {
+      in.intersect(new IntersectVisitor() {
+                     @Override
+                     public void visit(int docID) throws IOException {
+                       visitor.visit(docMap.oldToNew(docID));
+                     }
+
+                     @Override
+                     public void visit(int docID, byte[] packedValue) throws IOException {
+                       visitor.visit(docMap.oldToNew(docID), packedValue);
+                     }
+
+                     @Override
+                     public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                       return visitor.compare(minPackedValue, maxPackedValue);
+                     }
+                   });
+    }
+
+    @Override
+    public long estimatePointCount(IntersectVisitor visitor) {
+      return in.estimatePointCount(visitor);
+    }
+
+    @Override
+    public byte[] getMinPackedValue() throws IOException {
+      return in.getMinPackedValue();
+    }
+
+    @Override
+    public byte[] getMaxPackedValue() throws IOException {
+      return in.getMaxPackedValue();
+    }
+
+    @Override
+    public int getNumDimensions() throws IOException {
+      return in.getNumDimensions();
+    }
+
+    @Override
+    public int getNumIndexDimensions() throws IOException {
+      return in.getNumIndexDimensions();
+    }
+
+    @Override
+    public int getBytesPerDimension() throws IOException {
+      return in.getBytesPerDimension();
+    }
+
+    @Override
+    public long size() {
+      return in.size();
+    }
+
+    @Override
+    public int getDocCount() {
+      return in.getDocCount();
+    }
+  }
+
+
+
+
+
+  /** Return a sorted view of <code>reader</code> according to the order
+   *  defined by <code>sort</code>. If the reader is already sorted, this
+   *  method might return the reader as-is. */
+  public static CodecReader wrap(CodecReader reader, Sort sort) throws IOException {
+    return wrap(reader, new Sorter(sort).sort(reader), sort);
+  }
+
+  /** Expert: same as {@link #wrap(org.apache.lucene.index.CodecReader, Sort)} but operates directly on a {@link Sorter.DocMap}. */
+  static CodecReader wrap(CodecReader reader, Sorter.DocMap docMap, Sort sort) {
+    LeafMetaData metaData = reader.getMetaData();
+    LeafMetaData newMetaData = new LeafMetaData(metaData.getCreatedVersionMajor(), metaData.getMinVersion(), sort);
+    if (docMap == null) {
+      // the reader is already sorted
+      return new FilterCodecReader(reader) {
+        @Override
+        public CacheHelper getCoreCacheHelper() {
+          return null;
+        }
+
+        @Override
+        public CacheHelper getReaderCacheHelper() {
+          return null;
+        }
+
+        @Override
+        public LeafMetaData getMetaData() {
+          return newMetaData;
+        }
+
+        @Override
+        public String toString() {
+          return "SortingCodecReader(" + in + ")";
+        }
+      };
+    }
+    if (reader.maxDoc() != docMap.size()) {
+      throw new IllegalArgumentException("reader.maxDoc() should be equal to docMap.size(), got" + reader.maxDoc() + " != " + docMap.size());
+    }
+    assert Sorter.isConsistent(docMap);
+    return new SortingCodecReader(reader, docMap, newMetaData);
+  }
+
+  final Sorter.DocMap docMap; // pkg-protected to avoid synthetic accessor methods
+  final LeafMetaData metaData;
+
+  private SortingCodecReader(final CodecReader in, final Sorter.DocMap docMap, LeafMetaData metaData) {
+    super(in);
+    this.docMap = docMap;
+    this.metaData = metaData;
+  }
+
+
+  @Override
+  public FieldsProducer getPostingsReader() {
+    FieldsProducer postingsReader = in.getPostingsReader();
+    return new FieldsProducer() {
+      @Override
+      public void close() throws IOException {
+        postingsReader.close();
+      }
+
+      @Override
+      public void checkIntegrity() throws IOException {
+        postingsReader.checkIntegrity();
+      }
+
+      @Override
+      public Iterator<String> iterator() {
+        return postingsReader.iterator();
+      }
+
+      @Override
+      public Terms terms(String field) throws IOException {
+        Terms terms = postingsReader.terms(field);
+        return terms == null ? null : new FreqProxTermsWriter.SortingTerms(terms,
+            in.getFieldInfos().fieldInfo(field).getIndexOptions(), docMap);
+      }
+
+      @Override
+      public int size() {
+        return postingsReader.size();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return postingsReader.ramBytesUsed();
+      }
+    };
+  }
+
+  @Override
+  public StoredFieldsReader getFieldsReader() {
+    StoredFieldsReader delegate = in.getFieldsReader();
+    return newStoredFieldsReader(delegate);
+  }
+
+  private StoredFieldsReader newStoredFieldsReader(StoredFieldsReader delegate) {
+    return new StoredFieldsReader() {
+      @Override
+      public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException {
+        delegate.visitDocument(docMap.newToOld(docID), visitor);
+      }
+
+      @Override
+      public StoredFieldsReader clone() {
+        return newStoredFieldsReader(delegate.clone());
+      }
+
+      @Override
+      public void checkIntegrity() throws IOException {
+        delegate.checkIntegrity();
+      }
+
+      @Override
+      public void close() throws IOException {
+        delegate.close();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return delegate.ramBytesUsed();
+      }
+    };
+  }
+
+  @Override
+  public Bits getLiveDocs() {
+    final Bits inLiveDocs = in.getLiveDocs();
+    if (inLiveDocs == null) {
+      return null;
+    } else {
+      return new SortingBits(inLiveDocs, docMap);
+    }
+  }
+
+  @Override
+  public PointsReader getPointsReader() {
+    final PointsReader delegate = in.getPointsReader();
+    return new PointsReader() {
+      @Override
+      public void checkIntegrity() throws IOException {
+        delegate.checkIntegrity();
+      }
+
+      @Override
+      public PointValues getValues(String field) throws IOException {
+        return new SortingPointValues(delegate.getValues(field), docMap);
+      }
+
+      @Override
+      public void close() throws IOException {
+        delegate.close();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return delegate.ramBytesUsed();
+      }
+    };
+  }
+
+  private final Map<String, NumericDocValuesWriter.CachedNumericDVs> cachedNorms = new HashMap<>();
+
+  @Override
+  public NormsProducer getNormsReader() {
+    final NormsProducer delegate = in.getNormsReader();
+    return new NormsProducer() {
+      @Override
+      public NumericDocValues getNorms(FieldInfo field) throws IOException {
+        return produceNumericDocValues(field, delegate.getNorms(field), cachedNorms);
+      }
+
+      @Override
+      public void checkIntegrity() throws IOException {
+        delegate.checkIntegrity();
+      }
+
+      @Override
+      public void close() throws IOException {
+        delegate.close();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return delegate.ramBytesUsed();
+      }
+    };
+  }
+
+  private NumericDocValues produceNumericDocValues(FieldInfo field, NumericDocValues oldNorms,
+                                                   Map<String, NumericDocValuesWriter.CachedNumericDVs> cachedNorms) throws IOException {
+    NumericDocValuesWriter.CachedNumericDVs norms;
+    synchronized (cachedNorms) {
+      norms = cachedNorms.get(field);
+      if (norms == null) {
+        FixedBitSet docsWithField = new FixedBitSet(maxDoc());
+        long[] values = new long[maxDoc()];
+        while (true) {
+          int docID = oldNorms.nextDoc();
+          if (docID == NO_MORE_DOCS) {
+            break;
+          }
+          int newDocID = docMap.oldToNew(docID);
+          docsWithField.set(newDocID);
+          values[newDocID] = oldNorms.longValue();
+        }
+        norms = new NumericDocValuesWriter.CachedNumericDVs(values, docsWithField);
+        cachedNorms.put(field.name, norms);
+      }
+    }
+    return new NumericDocValuesWriter.SortingNumericDocValues(norms);
+  }
+
+  @Override
+  public DocValuesProducer getDocValuesReader() {
+    final DocValuesProducer delegate = in.getDocValuesReader();
+    return new DocValuesProducer() {
+      @Override
+      public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+        return produceNumericDocValues(field,delegate.getNumeric(field), cachedNumericDVs);
+      }
+
+      @Override
+      public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+        final BinaryDocValues oldDocValues = delegate.getBinary(field);
+        BinaryDocValuesWriter.CachedBinaryDVs dvs;
+        synchronized (cachedBinaryDVs) {
+          dvs = cachedBinaryDVs.get(field);
+          if (dvs == null) {
+            dvs = BinaryDocValuesWriter.sortDocValues(maxDoc(), docMap, oldDocValues);
+            cachedBinaryDVs.put(field.name, dvs);
+          }
+        }
+        return new BinaryDocValuesWriter.SortingBinaryDocValues(dvs);
+      }
+
+      @Override
+      public SortedDocValues getSorted(FieldInfo field) throws IOException {
+        SortedDocValues oldDocValues = delegate.getSorted(field);
+        int[] ords;
+        synchronized (cachedSortedDVs) {
+          ords = cachedSortedDVs.get(field);
+          if (ords == null) {
+            ords = new int[maxDoc()];
+            Arrays.fill(ords, -1);
+            int docID;
+            while ((docID = oldDocValues.nextDoc()) != NO_MORE_DOCS) {
+              int newDocID = docMap.oldToNew(docID);
+              ords[newDocID] = oldDocValues.ordValue();
+            }
+            cachedSortedDVs.put(field.name, ords);
+          }
+        }
+
+        return new SortedDocValuesWriter.SortingSortedDocValues(oldDocValues, ords);
+      }
+
+      @Override
+      public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+        final SortedNumericDocValues oldDocValues = delegate.getSortedNumeric(field);
+        long[][] values;
+        synchronized (cachedSortedNumericDVs) {
+          values = cachedSortedNumericDVs.get(field);
+          if (values == null) {
+            values = SortedNumericDocValuesWriter.sortDocValues(maxDoc(), docMap, oldDocValues);
+            cachedSortedNumericDVs.put(field.name, values);
+          }
+        }
+
+        return new SortedNumericDocValuesWriter.SortingSortedNumericDocValues(oldDocValues, values);
+      }
+
+      @Override
+      public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+        SortedSetDocValues oldDocValues = delegate.getSortedSet(field);
+        long[][] ords;
+        synchronized (cachedSortedSetDVs) {
+          ords = cachedSortedSetDVs.get(field);
+          if (ords == null) {
+            ords = SortedSetDocValuesWriter.sortDocValues(maxDoc(), docMap, oldDocValues);
+            cachedSortedSetDVs.put(field.name, ords);
+          }
+        }
+        return new SortedSetDocValuesWriter.SortingSortedSetDocValues(oldDocValues, ords);
+      }
+
+      @Override
+      public void checkIntegrity() throws IOException {
+        delegate.checkIntegrity();
+      }
+
+      @Override
+      public void close() throws IOException {
+        delegate.close();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return delegate.ramBytesUsed();
+      }
+    };
+  }
+
+  @Override
+  public TermVectorsReader getTermVectorsReader() {
+    return newTermVectorsReader(in.getTermVectorsReader());
+  }
+
+  private TermVectorsReader newTermVectorsReader(TermVectorsReader delegate) {
+    return new TermVectorsReader() {
+      @Override
+      public Fields get(int doc) throws IOException {
+        return delegate.get(docMap.newToOld(doc));
+      }
+
+      @Override
+      public void checkIntegrity() throws IOException {
+        delegate.checkIntegrity();
+      }
+
+      @Override
+      public TermVectorsReader clone() {
+        return newTermVectorsReader(delegate.clone());
+      }
+
+      @Override
+      public void close() throws IOException {
+        delegate.close();
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        return delegate.ramBytesUsed();
+      }
+    };
+  }
+
+  @Override
+  public String toString() {
+    return "SortingCodecReader(" + in + ")";
+  }
+
+  // no caching on sorted views
+  @Override
+  public CacheHelper getCoreCacheHelper() {
+    return null;
+  }
+
+  @Override
+  public CacheHelper getReaderCacheHelper() {
+    return null;
+  }
+
+  @Override
+  public LeafMetaData getMetaData() {
+    return metaData;
+  }
+
+}
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
deleted file mode 100644
index 5d71693..0000000
--- a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
+++ /dev/null
@@ -1,1268 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.index;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitSet;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.TimSorter;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
-
-import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
-import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
-
-/**
- * An {@link org.apache.lucene.index.LeafReader} which supports sorting documents by a given
- * {@link Sort}. This is package private and is only used by Lucene for BWC when it needs to merge
- * an unsorted flushed segment built by an older version (newly flushed segments are sorted since version 7.0).
- *
- * @lucene.experimental
- */
-class SortingLeafReader extends FilterLeafReader {
-
-  //TODO remove from here; move to FreqProxTermsWriter or FreqProxFields?
-  static class SortingFields extends FilterFields {
-
-    private final Sorter.DocMap docMap;
-    private final FieldInfos infos;
-
-    public SortingFields(final Fields in, FieldInfos infos, Sorter.DocMap docMap) {
-      super(in);
-      this.docMap = docMap;
-      this.infos = infos;
-    }
-
-    @Override
-    public Terms terms(final String field) throws IOException {
-      Terms terms = in.terms(field);
-      if (terms == null) {
-        return null;
-      } else {
-        return new SortingTerms(terms, infos.fieldInfo(field).getIndexOptions(), docMap);
-      }
-    }
-
-  }
-
-  private static class SortingTerms extends FilterTerms {
-
-    private final Sorter.DocMap docMap;
-    private final IndexOptions indexOptions;
-
-    public SortingTerms(final Terms in, IndexOptions indexOptions, final Sorter.DocMap docMap) {
-      super(in);
-      this.docMap = docMap;
-      this.indexOptions = indexOptions;
-    }
-
-    @Override
-    public TermsEnum iterator() throws IOException {
-      return new SortingTermsEnum(in.iterator(), docMap, indexOptions, hasPositions());
-    }
-
-    @Override
-    public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm)
-        throws IOException {
-      return new SortingTermsEnum(in.intersect(compiled, startTerm), docMap, indexOptions, hasPositions());
-    }
-
-  }
-
-  private static class SortingTermsEnum extends FilterTermsEnum {
-
-    final Sorter.DocMap docMap; // pkg-protected to avoid synthetic accessor methods
-    private final IndexOptions indexOptions;
-    private final boolean hasPositions;
-
-    public SortingTermsEnum(final TermsEnum in, Sorter.DocMap docMap, IndexOptions indexOptions, boolean hasPositions) {
-      super(in);
-      this.docMap = docMap;
-      this.indexOptions = indexOptions;
-      this.hasPositions = hasPositions;
-    }
-
-    @Override
-    public PostingsEnum postings( PostingsEnum reuse, final int flags) throws IOException {
-
-      if (hasPositions && PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS)) {
-        final PostingsEnum inReuse;
-        final SortingPostingsEnum wrapReuse;
-        if (reuse != null && reuse instanceof SortingPostingsEnum) {
-          // if we're asked to reuse the given DocsEnum and it is Sorting, return
-          // the wrapped one, since some Codecs expect it.
-          wrapReuse = (SortingPostingsEnum) reuse;
-          inReuse = wrapReuse.getWrapped();
-        } else {
-          wrapReuse = null;
-          inReuse = reuse;
-        }
-
-        final PostingsEnum inDocsAndPositions = in.postings(inReuse, flags);
-        // we ignore the fact that offsets may be stored but not asked for,
-        // since this code is expected to be used during addIndexes which will
-        // ask for everything. if that assumption changes in the future, we can
-        // factor in whether 'flags' says offsets are not required.
-        final boolean storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-        return new SortingPostingsEnum(docMap.size(), wrapReuse, inDocsAndPositions, docMap, storeOffsets);
-      }
-
-      final PostingsEnum inReuse;
-      final SortingDocsEnum wrapReuse;
-      if (reuse != null && reuse instanceof SortingDocsEnum) {
-        // if we're asked to reuse the given DocsEnum and it is Sorting, return
-        // the wrapped one, since some Codecs expect it.
-        wrapReuse = (SortingDocsEnum) reuse;
-        inReuse = wrapReuse.getWrapped();
-      } else {
-        wrapReuse = null;
-        inReuse = reuse;
-      }
-
-      final PostingsEnum inDocs = in.postings(inReuse, flags);
-      final boolean withFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >=0 && PostingsEnum.featureRequested(flags, PostingsEnum.FREQS);
-      return new SortingDocsEnum(docMap.size(), wrapReuse, inDocs, withFreqs, docMap);
-    }
-
-  }
-
-  static class SortingBinaryDocValues extends BinaryDocValues {
-
-    private final CachedBinaryDVs dvs;
-    private int docID = -1;
-
-    public SortingBinaryDocValues(CachedBinaryDVs dvs) {
-      this.dvs = dvs;
-    }
-
-    @Override
-    public int nextDoc() {
-      if (docID+1 == dvs.docsWithField.length()) {
-        docID = NO_MORE_DOCS;
-      } else {
-        docID = dvs.docsWithField.nextSetBit(docID+1);
-      }
-
-      return docID;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int advance(int target) {
-      docID = dvs.docsWithField.nextSetBit(target);
-      return docID;
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      docID = target;
-      return dvs.docsWithField.get(target);
-    }
-
-    @Override
-    public BytesRef binaryValue() {
-      return dvs.values[docID];
-    }
-
-    @Override
-    public long cost() {
-      return dvs.docsWithField.cardinality();
-    }
-  }
-
-  private final Map<String,CachedNumericDVs> cachedNumericDVs = new HashMap<>();
-
-  static class CachedNumericDVs {
-    private final long[] values;
-    private final BitSet docsWithField;
-
-    public CachedNumericDVs(long[] values, BitSet docsWithField) {
-      this.values = values;
-      this.docsWithField = docsWithField;
-    }
-  }
-
-  private final Map<String,CachedBinaryDVs> cachedBinaryDVs = new HashMap<>();
-
-  static class CachedBinaryDVs {
-    // TODO: at least cutover to BytesRefArray here:
-    private final BytesRef[] values;
-    private final BitSet docsWithField;
-
-    public CachedBinaryDVs(BytesRef[] values, BitSet docsWithField) {
-      this.values = values;
-      this.docsWithField = docsWithField;
-    }
-  }
-
-  private final Map<String,int[]> cachedSortedDVs = new HashMap<>();
-
-  static class SortingNumericDocValues extends NumericDocValues {
-
-    private final CachedNumericDVs dvs;
-    private int docID = -1;
-
-    public SortingNumericDocValues(CachedNumericDVs dvs) {
-      this.dvs = dvs;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int nextDoc() {
-      if (docID+1 == dvs.docsWithField.length()) {
-        docID = NO_MORE_DOCS;
-      } else {
-        docID = dvs.docsWithField.nextSetBit(docID+1);
-      }
-
-      return docID;
-    }
-
-    @Override
-    public int advance(int target) {
-      docID = dvs.docsWithField.nextSetBit(target);
-      return docID;
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      docID = target;
-      return dvs.docsWithField.get(target);
-    }
-
-    @Override
-    public long longValue() {
-      return dvs.values[docID];
-    }
-
-    @Override
-    public long cost() {
-      return dvs.docsWithField.cardinality();
-    }
-  }
-
-  private static class SortingBits implements Bits {
-
-    private final Bits in;
-    private final Sorter.DocMap docMap;
-
-    public SortingBits(final Bits in, Sorter.DocMap docMap) {
-      this.in = in;
-      this.docMap = docMap;
-    }
-
-    @Override
-    public boolean get(int index) {
-      return in.get(docMap.newToOld(index));
-    }
-
-    @Override
-    public int length() {
-      return in.length();
-    }
-  }
-
-  private static class SortingPointValues extends PointValues {
-
-    private final PointValues in;
-    private final Sorter.DocMap docMap;
-
-    public SortingPointValues(final PointValues in, Sorter.DocMap docMap) {
-      this.in = in;
-      this.docMap = docMap;
-    }
-
-    @Override
-    public void intersect(IntersectVisitor visitor) throws IOException {
-      in.intersect(new IntersectVisitor() {
-                     @Override
-                     public void visit(int docID) throws IOException {
-                       visitor.visit(docMap.oldToNew(docID));
-                     }
-
-                     @Override
-                     public void visit(int docID, byte[] packedValue) throws IOException {
-                       visitor.visit(docMap.oldToNew(docID), packedValue);
-                     }
-
-                     @Override
-                     public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-                       return visitor.compare(minPackedValue, maxPackedValue);
-                     }
-                   });
-    }
-
-    @Override
-    public long estimatePointCount(IntersectVisitor visitor) {
-      return in.estimatePointCount(visitor);
-    }
-
-    @Override
-    public byte[] getMinPackedValue() throws IOException {
-      return in.getMinPackedValue();
-    }
-
-    @Override
-    public byte[] getMaxPackedValue() throws IOException {
-      return in.getMaxPackedValue();
-    }
-
-    @Override
-    public int getNumDimensions() throws IOException {
-      return in.getNumDimensions();
-    }
-
-    @Override
-    public int getNumIndexDimensions() throws IOException {
-      return in.getNumIndexDimensions();
-    }
-
-    @Override
-    public int getBytesPerDimension() throws IOException {
-      return in.getBytesPerDimension();
-    }
-
-    @Override
-    public long size() {
-      return in.size();
-    }
-
-    @Override
-    public int getDocCount() {
-      return in.getDocCount();
-    }
-  }
-
-  static class SortingSortedDocValues extends SortedDocValues {
-
-    private final SortedDocValues in;
-    private final int[] ords;
-    private int docID = -1;
-
-    SortingSortedDocValues(SortedDocValues in, int[] ords) {
-      this.in = in;
-      this.ords = ords;
-      assert ords != null;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int nextDoc() {
-      while (true) {
-        docID++;
-        if (docID == ords.length) {
-          docID = NO_MORE_DOCS;
-          break;
-        }
-        if (ords[docID] != -1) {
-          break;
-        }
-        // skip missing docs
-      }
-
-      return docID;
-    }
-
-    @Override
-    public int advance(int target) {
-      if (target >= ords.length) {
-        docID = NO_MORE_DOCS;
-      } else {
-        docID = target;
-        if (ords[docID] == -1) {
-          nextDoc();
-        }
-      }
-      return docID;
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      docID = target;
-      return ords[target] != -1;
-    }
-
-    @Override
-    public int ordValue() {
-      return ords[docID];
-    }
-
-    @Override
-    public long cost() {
-      return in.cost();
-    }
-
-    @Override
-    public BytesRef lookupOrd(int ord) throws IOException {
-      return in.lookupOrd(ord);
-    }
-
-    @Override
-    public int getValueCount() {
-      return in.getValueCount();
-    }
-  }
-
-  // TODO: pack long[][] into an int[] (offset) and long[] instead:
-  private final Map<String,long[][]> cachedSortedSetDVs = new HashMap<>();
-
-  static class SortingSortedSetDocValues extends SortedSetDocValues {
-
-    private final SortedSetDocValues in;
-    private final long[][] ords;
-    private int docID = -1;
-    private int ordUpto;
-
-    SortingSortedSetDocValues(SortedSetDocValues in, long[][] ords) {
-      this.in = in;
-      this.ords = ords;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int nextDoc() {
-      while (true) {
-        docID++;
-        if (docID == ords.length) {
-          docID = NO_MORE_DOCS;
-          break;
-        }
-        if (ords[docID] != null) {
-          break;
-        }
-        // skip missing docs
-      }
-      ordUpto = 0;
-      return docID;
-    }
-
-    @Override
-    public int advance(int target) {
-      if (target >= ords.length) {
-        docID = NO_MORE_DOCS;
-      } else {
-        docID = target;
-        if (ords[docID] == null) {
-          nextDoc();
-        } else {
-          ordUpto = 0;
-        }
-      }
-      return docID;
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      docID = target;
-      ordUpto = 0;
-      return ords[docID] != null;
-    }
-
-    @Override
-    public long nextOrd() {
-      if (ordUpto == ords[docID].length) {
-        return NO_MORE_ORDS;
-      } else {
-        return ords[docID][ordUpto++];
-      }
-    }
-
-    @Override
-    public long cost() {
-      return in.cost();
-    }
-
-    @Override
-    public BytesRef lookupOrd(long ord) throws IOException {
-      return in.lookupOrd(ord);
-    }
-
-    @Override
-    public long getValueCount() {
-      return in.getValueCount();
-    }
-  }
-
-  private final Map<String,long[][]> cachedSortedNumericDVs = new HashMap<>();
-
-  static class SortingSortedNumericDocValues extends SortedNumericDocValues {
-    private final SortedNumericDocValues in;
-    private final long[][] values;
-    private int docID = -1;
-    private int upto;
-
-    SortingSortedNumericDocValues(SortedNumericDocValues in, long[][] values) {
-      this.in = in;
-      this.values = values;
-    }
-
-    @Override
-    public int docID() {
-      return docID;
-    }
-
-    @Override
-    public int nextDoc() {
-      while (true) {
-        docID++;
-        if (docID == values.length) {
-          docID = NO_MORE_DOCS;
-          break;
-        }
-        if (values[docID] != null) {
-          break;
-        }
-        // skip missing docs
-      }
-      upto = 0;
-      return docID;
-    }
-
-    @Override
-    public int advance(int target) {
-      if (target >= values.length) {
-        docID = NO_MORE_DOCS;
-        return docID;
-      } else {
-        docID = target-1;
-        return nextDoc();
-      }
-    }
-
-    @Override
-    public boolean advanceExact(int target) throws IOException {
-      docID = target;
-      upto = 0;
-      return values[docID] != null;
-    }
-
-    @Override
-    public long nextValue() {
-      if (upto == values[docID].length) {
-        throw new AssertionError();
-      } else {
-        return values[docID][upto++];
-      }
-    }
-
-    @Override
-    public long cost() {
-      return in.cost();
-    }
-
-    @Override
-    public int docValueCount() {
-      return values[docID].length;
-    }
-  }
-
-  static class SortingDocsEnum extends FilterPostingsEnum {
-
-    private static final class DocFreqSorter extends TimSorter {
-
-      private int[] docs;
-      private int[] freqs;
-      private final int[] tmpDocs;
-      private int[] tmpFreqs;
-
-      public DocFreqSorter(int maxDoc) {
-        super(maxDoc / 64);
-        this.tmpDocs = new int[maxDoc / 64];
-      }
-
-      public void reset(int[] docs, int[] freqs) {
-        this.docs = docs;
-        this.freqs = freqs;
-        if (freqs != null && tmpFreqs == null) {
-          tmpFreqs = new int[tmpDocs.length];
-        }
-      }
-
-      @Override
-      protected int compare(int i, int j) {
-        return docs[i] - docs[j];
-      }
-
-      @Override
-      protected void swap(int i, int j) {
-        int tmpDoc = docs[i];
-        docs[i] = docs[j];
-        docs[j] = tmpDoc;
-
-        if (freqs != null) {
-          int tmpFreq = freqs[i];
-          freqs[i] = freqs[j];
-          freqs[j] = tmpFreq;
-        }
-      }
-
-      @Override
-      protected void copy(int src, int dest) {
-        docs[dest] = docs[src];
-        if (freqs != null) {
-          freqs[dest] = freqs[src];
-        }
-      }
-
-      @Override
-      protected void save(int i, int len) {
-        System.arraycopy(docs, i, tmpDocs, 0, len);
-        if (freqs != null) {
-          System.arraycopy(freqs, i, tmpFreqs, 0, len);
-        }
-      }
-
-      @Override
-      protected void restore(int i, int j) {
-        docs[j] = tmpDocs[i];
-        if (freqs != null) {
-          freqs[j] = tmpFreqs[i];
-        }
-      }
-
-      @Override
-      protected int compareSaved(int i, int j) {
-        return tmpDocs[i] - docs[j];
-      }
-    }
-
-    private final int maxDoc;
-    private final DocFreqSorter sorter;
-    private int[] docs;
-    private int[] freqs;
-    private int docIt = -1;
-    private final int upto;
-    private final boolean withFreqs;
-
-    SortingDocsEnum(int maxDoc, SortingDocsEnum reuse, final PostingsEnum in, boolean withFreqs, final Sorter.DocMap docMap) throws IOException {
-      super(in);
-      this.maxDoc = maxDoc;
-      this.withFreqs = withFreqs;
-      if (reuse != null) {
-        if (reuse.maxDoc == maxDoc) {
-          sorter = reuse.sorter;
-        } else {
-          sorter = new DocFreqSorter(maxDoc);
-        }
-        docs = reuse.docs;
-        freqs = reuse.freqs; // maybe null
-      } else {
-        docs = new int[64];
-        sorter = new DocFreqSorter(maxDoc);
-      }
-      docIt = -1;
-      int i = 0;
-      int doc;
-      if (withFreqs) {
-        if (freqs == null || freqs.length < docs.length) {
-          freqs = new int[docs.length];
-        }
-        while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS){
-          if (i >= docs.length) {
-            docs = ArrayUtil.grow(docs, docs.length + 1);
-            freqs = ArrayUtil.grow(freqs, freqs.length + 1);
-          }
-          docs[i] = docMap.oldToNew(doc);
-          freqs[i] = in.freq();
-          ++i;
-        }
-      } else {
-        freqs = null;
-        while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS){
-          if (i >= docs.length) {
-            docs = ArrayUtil.grow(docs, docs.length + 1);
-          }
-          docs[i++] = docMap.oldToNew(doc);
-        }
-      }
-      // TimSort can save much time compared to other sorts in case of
-      // reverse sorting, or when sorting a concatenation of sorted readers
-      sorter.reset(docs, freqs);
-      sorter.sort(0, i);
-      upto = i;
-    }
-
-    // for testing
-    boolean reused(PostingsEnum other) {
-      if (other == null || !(other instanceof SortingDocsEnum)) {
-        return false;
-      }
-      return docs == ((SortingDocsEnum) other).docs;
-    }
-
-    @Override
-    public int advance(final int target) throws IOException {
-      // need to support it for checkIndex, but in practice it won't be called, so
-      // don't bother to implement efficiently for now.
-      return slowAdvance(target);
-    }
-
-    @Override
-    public int docID() {
-      return docIt < 0 ? -1 : docIt >= upto ? NO_MORE_DOCS : docs[docIt];
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return withFreqs && docIt < upto ? freqs[docIt] : 1;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      if (++docIt >= upto) return NO_MORE_DOCS;
-      return docs[docIt];
-    }
-
-    /** Returns the wrapped {@link PostingsEnum}. */
-    PostingsEnum getWrapped() {
-      return in;
-    }
-    
-    // we buffer up docs/freqs only, don't forward any positions requests to underlying enum
-
-    @Override
-    public int nextPosition() throws IOException {
-      return -1;
-    }
-
-    @Override
-    public int startOffset() throws IOException {
-      return -1;
-    }
-
-    @Override
-    public int endOffset() throws IOException {
-      return -1;
-    }
-
-    @Override
-    public BytesRef getPayload() throws IOException {
-      return null;
-    }
-  }
-
-  static class SortingPostingsEnum extends FilterPostingsEnum {
-
-    /**
-     * A {@link TimSorter} which sorts two parallel arrays of doc IDs and
-     * offsets in one go. Everytime a doc ID is 'swapped', its corresponding offset
-     * is swapped too.
-     */
-    private static final class DocOffsetSorter extends TimSorter {
-
-      private int[] docs;
-      private long[] offsets;
-      private final int[] tmpDocs;
-      private final long[] tmpOffsets;
-
-      public DocOffsetSorter(int maxDoc) {
-        super(maxDoc / 64);
-        this.tmpDocs = new int[maxDoc / 64];
-        this.tmpOffsets = new long[maxDoc / 64];
-      }
-
-      public void reset(int[] docs, long[] offsets) {
-        this.docs = docs;
-        this.offsets = offsets;
-      }
-
-      @Override
-      protected int compare(int i, int j) {
-        return docs[i] - docs[j];
-      }
-
-      @Override
-      protected void swap(int i, int j) {
-        int tmpDoc = docs[i];
-        docs[i] = docs[j];
-        docs[j] = tmpDoc;
-
-        long tmpOffset = offsets[i];
-        offsets[i] = offsets[j];
-        offsets[j] = tmpOffset;
-      }
-
-      @Override
-      protected void copy(int src, int dest) {
-        docs[dest] = docs[src];
-        offsets[dest] = offsets[src];
-      }
-
-      @Override
-      protected void save(int i, int len) {
-        System.arraycopy(docs, i, tmpDocs, 0, len);
-        System.arraycopy(offsets, i, tmpOffsets, 0, len);
-      }
-
-      @Override
-      protected void restore(int i, int j) {
-        docs[j] = tmpDocs[i];
-        offsets[j] = tmpOffsets[i];
-      }
-
-      @Override
-      protected int compareSaved(int i, int j) {
-        return tmpDocs[i] - docs[j];
-      }
-    }
-
-    private final int maxDoc;
-    private final DocOffsetSorter sorter;
-    private int[] docs;
-    private long[] offsets;
-    private final int upto;
-
-    private final IndexInput postingInput;
-    private final boolean storeOffsets;
-
-    private int docIt = -1;
-    private int pos;
-    private int startOffset = -1;
-    private int endOffset = -1;
-    private final BytesRef payload;
-    private int currFreq;
-
-    private final RAMFile file;
-
-    SortingPostingsEnum(int maxDoc, SortingPostingsEnum reuse, final PostingsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
-      super(in);
-      this.maxDoc = maxDoc;
-      this.storeOffsets = storeOffsets;
-      if (reuse != null) {
-        docs = reuse.docs;
-        offsets = reuse.offsets;
-        payload = reuse.payload;
-        file = reuse.file;
-        if (reuse.maxDoc == maxDoc) {
-          sorter = reuse.sorter;
-        } else {
-          sorter = new DocOffsetSorter(maxDoc);
-        }
-      } else {
-        docs = new int[32];
-        offsets = new long[32];
-        payload = new BytesRef(32);
-        file = new RAMFile();
-        sorter = new DocOffsetSorter(maxDoc);
-      }
-      final IndexOutput out = new RAMOutputStream(file, false);
-      int doc;
-      int i = 0;
-      while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        if (i == docs.length) {
-          final int newLength = ArrayUtil.oversize(i + 1, 4);
-          docs = ArrayUtil.growExact(docs, newLength);
-          offsets = ArrayUtil.growExact(offsets, newLength);
-        }
-        docs[i] = docMap.oldToNew(doc);
-        offsets[i] = out.getFilePointer();
-        addPositions(in, out);
-        i++;
-      }
-      upto = i;
-      sorter.reset(docs, offsets);
-      sorter.sort(0, upto);
-      out.close();
-      this.postingInput = new RAMInputStream("", file);
-    }
-
-    // for testing
-    boolean reused(PostingsEnum other) {
-      if (other == null || !(other instanceof SortingPostingsEnum)) {
-        return false;
-      }
-      return docs == ((SortingPostingsEnum) other).docs;
-    }
-
-    private void addPositions(final PostingsEnum in, final IndexOutput out) throws IOException {
-      int freq = in.freq();
-      out.writeVInt(freq);
-      int previousPosition = 0;
-      int previousEndOffset = 0;
-      for (int i = 0; i < freq; i++) {
-        final int pos = in.nextPosition();
-        final BytesRef payload = in.getPayload();
-        // The low-order bit of token is set only if there is a payload, the
-        // previous bits are the delta-encoded position.
-        final int token = (pos - previousPosition) << 1 | (payload == null ? 0 : 1);
-        out.writeVInt(token);
-        previousPosition = pos;
-        if (storeOffsets) { // don't encode offsets if they are not stored
-          final int startOffset = in.startOffset();
-          final int endOffset = in.endOffset();
-          out.writeVInt(startOffset - previousEndOffset);
-          out.writeVInt(endOffset - startOffset);
-          previousEndOffset = endOffset;
-        }
-        if (payload != null) {
-          out.writeVInt(payload.length);
-          out.writeBytes(payload.bytes, payload.offset, payload.length);
-        }
-      }
-    }
-
-    @Override
-    public int advance(final int target) throws IOException {
-      // need to support it for checkIndex, but in practice it won't be called, so
-      // don't bother to implement efficiently for now.
-      return slowAdvance(target);
-    }
-
-    @Override
-    public int docID() {
-      return docIt < 0 ? -1 : docIt >= upto ? NO_MORE_DOCS : docs[docIt];
-    }
-
-    @Override
-    public int endOffset() throws IOException {
-      return endOffset;
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return currFreq;
-    }
-
-    @Override
-    public BytesRef getPayload() throws IOException {
-      return payload.length == 0 ? null : payload;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      if (++docIt >= upto) return DocIdSetIterator.NO_MORE_DOCS;
-      postingInput.seek(offsets[docIt]);
-      currFreq = postingInput.readVInt();
-      // reset variables used in nextPosition
-      pos = 0;
-      endOffset = 0;
-      return docs[docIt];
-    }
-
-    @Override
-    public int nextPosition() throws IOException {
-      final int token = postingInput.readVInt();
-      pos += token >>> 1;
-      if (storeOffsets) {
-        startOffset = endOffset + postingInput.readVInt();
-        endOffset = startOffset + postingInput.readVInt();
-      }
-      if ((token & 1) != 0) {
-        payload.offset = 0;
-        payload.length = postingInput.readVInt();
-        if (payload.length > payload.bytes.length) {
-          payload.bytes = new byte[ArrayUtil.oversize(payload.length, 1)];
-        }
-        postingInput.readBytes(payload.bytes, 0, payload.length);
-      } else {
-        payload.length = 0;
-      }
-      return pos;
-    }
-
-    @Override
-    public int startOffset() throws IOException {
-      return startOffset;
-    }
-
-    /** Returns the wrapped {@link PostingsEnum}. */
-    PostingsEnum getWrapped() {
-      return in;
-    }
-  }
-
-  /** Return a sorted view of <code>reader</code> according to the order
-   *  defined by <code>sort</code>. If the reader is already sorted, this
-   *  method might return the reader as-is. */
-  public static LeafReader wrap(LeafReader reader, Sort sort) throws IOException {
-    return wrap(reader, new Sorter(sort).sort(reader));
-  }
-
-  /** Expert: same as {@link #wrap(org.apache.lucene.index.LeafReader, Sort)} but operates directly on a {@link Sorter.DocMap}. */
-  static LeafReader wrap(LeafReader reader, Sorter.DocMap docMap) {
-    if (docMap == null) {
-      // the reader is already sorted
-      return reader;
-    }
-    if (reader.maxDoc() != docMap.size()) {
-      throw new IllegalArgumentException("reader.maxDoc() should be equal to docMap.size(), got" + reader.maxDoc() + " != " + docMap.size());
-    }
-    assert Sorter.isConsistent(docMap);
-    return new SortingLeafReader(reader, docMap);
-  }
-
-  final Sorter.DocMap docMap; // pkg-protected to avoid synthetic accessor methods
-
-  private SortingLeafReader(final LeafReader in, final Sorter.DocMap docMap) {
-    super(in);
-    this.docMap = docMap;
-  }
-
-  @Override
-  public void document(final int docID, final StoredFieldVisitor visitor) throws IOException {
-    in.document(docMap.newToOld(docID), visitor);
-  }
-
-  @Override
-  public Terms terms(String field) throws IOException {
-    Terms terms = super.terms(field);
-    return terms==null ? null : new SortingTerms(terms, in.getFieldInfos().fieldInfo(field).getIndexOptions(), docMap);
-  }
-
-  @Override
-  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
-    final BinaryDocValues oldDocValues = in.getBinaryDocValues(field);
-    if (oldDocValues == null) return null;
-    CachedBinaryDVs dvs;
-    synchronized (cachedBinaryDVs) {
-      dvs = cachedBinaryDVs.get(field);
-      if (dvs == null) {
-        FixedBitSet docsWithField = new FixedBitSet(maxDoc());
-        BytesRef[] values = new BytesRef[maxDoc()];
-        while (true) {
-          int docID = oldDocValues.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            break;
-          }
-          int newDocID = docMap.oldToNew(docID);
-          docsWithField.set(newDocID);
-          values[newDocID] = BytesRef.deepCopyOf(oldDocValues.binaryValue());
-        }
-        dvs = new CachedBinaryDVs(values, docsWithField);
-        cachedBinaryDVs.put(field, dvs);
-      }
-    }
-    return new SortingBinaryDocValues(dvs);
-  }
-  
-
-  @Override
-  public Bits getLiveDocs() {
-    final Bits inLiveDocs = in.getLiveDocs();
-    if (inLiveDocs == null) {
-      return null;
-    } else {
-      return new SortingBits(inLiveDocs, docMap);
-    }
-  }
-
-  @Override
-  public PointValues getPointValues(String fieldName) throws IOException {
-    final PointValues inPointValues = in.getPointValues(fieldName);
-    if (inPointValues == null) {
-      return null;
-    } else {
-      return new SortingPointValues(inPointValues, docMap);
-    }
-  }
-
-  private final Map<String,CachedNumericDVs> cachedNorms = new HashMap<>();
-
-  @Override
-  public NumericDocValues getNormValues(String field) throws IOException {
-    final NumericDocValues oldNorms = in.getNormValues(field);
-    if (oldNorms == null) return null;
-    CachedNumericDVs norms;
-    synchronized (cachedNorms) {
-      norms = cachedNorms.get(field);
-      if (norms == null) {
-        FixedBitSet docsWithField = new FixedBitSet(maxDoc());
-        long[] values = new long[maxDoc()];
-        while (true) {
-          int docID = oldNorms.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            break;
-          }
-          int newDocID = docMap.oldToNew(docID);
-          docsWithField.set(newDocID);
-          values[newDocID] = oldNorms.longValue();
-        }
-        norms = new CachedNumericDVs(values, docsWithField);
-        cachedNorms.put(field, norms);
-      }
-    }
-    return new SortingNumericDocValues(norms);
-  }
-
-  @Override
-  public NumericDocValues getNumericDocValues(String field) throws IOException {
-    final NumericDocValues oldDocValues = in.getNumericDocValues(field);
-    if (oldDocValues == null) return null;
-    CachedNumericDVs dvs;
-    synchronized (cachedNumericDVs) {
-      dvs = cachedNumericDVs.get(field);
-      if (dvs == null) {
-        FixedBitSet docsWithField = new FixedBitSet(maxDoc());
-        long[] values = new long[maxDoc()];
-        while (true) {
-          int docID = oldDocValues.nextDoc();
-          if (docID == NO_MORE_DOCS) {
-            break;
-          }
-          int newDocID = docMap.oldToNew(docID);
-          docsWithField.set(newDocID);
-          values[newDocID] = oldDocValues.longValue();
-        }
-        dvs = new CachedNumericDVs(values, docsWithField);
-        cachedNumericDVs.put(field, dvs);
-      }
-    }
-    return new SortingNumericDocValues(dvs);
-  }
-
-  @Override
-  public SortedNumericDocValues getSortedNumericDocValues(String field)
-      throws IOException {
-    final SortedNumericDocValues oldDocValues = in.getSortedNumericDocValues(field);
-    if (oldDocValues == null) {
-      return null;
-    }
-
-    long[][] values;
-    synchronized (cachedSortedNumericDVs) {
-      values = cachedSortedNumericDVs.get(field);
-      if (values == null) {
-        values = new long[maxDoc()][];
-        int docID;
-        while ((docID = oldDocValues.nextDoc()) != NO_MORE_DOCS) {
-          int newDocID = docMap.oldToNew(docID);
-          long[] docValues = new long[oldDocValues.docValueCount()];
-          for(int i=0;i<docValues.length;i++) {
-            docValues[i] = oldDocValues.nextValue();
-          }
-          values[newDocID] = docValues;
-        }
-        cachedSortedNumericDVs.put(field, values);
-      }
-    }
-
-    return new SortingSortedNumericDocValues(oldDocValues, values);
-  }
-
-  @Override
-  public SortedDocValues getSortedDocValues(String field) throws IOException {
-    SortedDocValues oldDocValues = in.getSortedDocValues(field);
-    if (oldDocValues == null) {
-      return null;
-    }
-
-    int[] ords;
-    synchronized (cachedSortedDVs) {
-      ords = cachedSortedDVs.get(field);
-      if (ords == null) {
-        ords = new int[maxDoc()];
-        Arrays.fill(ords, -1);
-        int docID;
-        while ((docID = oldDocValues.nextDoc()) != NO_MORE_DOCS) {
-          int newDocID = docMap.oldToNew(docID);
-          ords[newDocID] = oldDocValues.ordValue();
-        }
-        cachedSortedDVs.put(field, ords);
-      }
-    }
-    
-    return new SortingSortedDocValues(oldDocValues, ords);
-  }
-
-  @Override
-  public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
-    SortedSetDocValues oldDocValues = in.getSortedSetDocValues(field);
-    if (oldDocValues == null) {
-      return null;
-    }
-
-    long[][] ords;
-    synchronized (cachedSortedSetDVs) {
-      ords = cachedSortedSetDVs.get(field);
-      if (ords == null) {
-        ords = new long[maxDoc()][];
-        int docID;
-        while ((docID = oldDocValues.nextDoc()) != NO_MORE_DOCS) {
-          int newDocID = docMap.oldToNew(docID);
-          long[] docOrds = new long[1];
-          int upto = 0;
-          while (true) {
-            long ord = oldDocValues.nextOrd();
-            if (ord == NO_MORE_ORDS) {
-              break;
-            }
-            if (upto == docOrds.length) {
-              docOrds = ArrayUtil.grow(docOrds);
-            }
-            docOrds[upto++] = ord;
-          }
-          ords[newDocID] = ArrayUtil.copyOfSubArray(docOrds, 0, upto);
-        }
-        cachedSortedSetDVs.put(field, ords);
-      }
-    }
-
-    return new SortingSortedSetDocValues(oldDocValues, ords);
-  }
-
-  @Override
-  public Fields getTermVectors(final int docID) throws IOException {
-    return in.getTermVectors(docMap.newToOld(docID));
-  }
-
-  @Override
-  public String toString() {
-    return "SortingLeafReader(" + in + ")";
-  }
-
-  // no caching on sorted views
-
-  @Override
-  public CacheHelper getCoreCacheHelper() {
-    return null;
-  }
-
-  @Override
-  public CacheHelper getReaderCacheHelper() {
-    return null;
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSortingCodecReader.java b/lucene/core/src/test/org/apache/lucene/index/TestSortingCodecReader.java
new file mode 100644
index 0000000..c1ac197
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSortingCodecReader.java
@@ -0,0 +1,224 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+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.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestSortingCodecReader extends LuceneTestCase {
+
+  public void testSortOnAddIndicesInt() throws IOException {
+    Directory tmpDir = newDirectory();
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    IndexWriter w = new IndexWriter(tmpDir, iwc);
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("foo", 18));
+    w.addDocument(doc);
+
+
+    doc = new Document();
+    doc.add(new NumericDocValuesField("foo", -1));
+    w.addDocument(doc);
+    w.commit();
+
+    doc = new Document();
+    doc.add(new NumericDocValuesField("foo", 7));
+    w.addDocument(doc);
+    w.commit();
+    w.close();
+    Sort indexSort = new Sort(new SortField("foo", SortField.Type.INT));
+
+    iwc = new IndexWriterConfig(new MockAnalyzer(random())).setIndexSort(indexSort);
+    w = new IndexWriter(dir, iwc);
+    try (DirectoryReader reader = DirectoryReader.open(tmpDir)) {
+      List<CodecReader> readers = new ArrayList<>();
+      for (LeafReaderContext ctx : reader.leaves()) {
+        CodecReader wrap = SortingCodecReader.wrap(SlowCodecReaderWrapper.wrap(ctx.reader()), indexSort);
+        assertTrue(wrap.toString(), wrap.toString().startsWith("SortingCodecReader("));
+        readers.add(wrap);
+
+      }
+      w.addIndexes(readers.toArray(new CodecReader[0]));
+    }
+    DirectoryReader r = DirectoryReader.open(w);
+    LeafReader leaf = getOnlyLeafReader(r);
+    assertEquals(3, leaf.maxDoc());
+    NumericDocValues values = leaf.getNumericDocValues("foo");
+    assertEquals(0, values.nextDoc());
+    assertEquals(-1, values.longValue());
+    assertEquals(1, values.nextDoc());
+    assertEquals(7, values.longValue());
+    assertEquals(2, values.nextDoc());
+    assertEquals(18, values.longValue());
+    assertNotNull(leaf.getMetaData().getSort());
+    IOUtils.close(r, w, dir, tmpDir);
+  }
+
+  public void testSortOnAddIndicesRandom() throws IOException {
+    try (Directory dir = newDirectory()) {
+      int numDocs = atLeast(200);
+      int actualNumDocs;
+      List<Integer> docIds = new ArrayList<>(numDocs);
+      for (int i = 0; i < numDocs; i++) {
+        docIds.add(i);
+      }
+      Collections.shuffle(docIds, random());
+      try (RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) {
+        for (int i = 0; i < numDocs; i++) {
+          int docId = docIds.get(i);
+          Document doc = new Document();
+          doc.add(new NumericDocValuesField("foo", random().nextInt(20)));
+          doc.add(new StringField("id", Integer.toString(docId), Field.Store.YES));
+          doc.add(new LongPoint("id", docId));
+          doc.add(new TextField("text_field", RandomStrings.randomRealisticUnicodeOfLength(random(), 25), Field.Store.YES));
+          doc.add(new SortedNumericDocValuesField("sorted_numeric_dv", docId));
+          doc.add(new SortedDocValuesField("binary_sorted_dv", new BytesRef(Integer.toString(docId))));
+          doc.add(new BinaryDocValuesField("binary_dv", new BytesRef(Integer.toString(docId))));
+          doc.add(new SortedSetDocValuesField("sorted_set_dv", new BytesRef(Integer.toString(docId))));
+
+          FieldType ft = new FieldType(StringField.TYPE_NOT_STORED);
+          ft.setStoreTermVectors(true);
+          doc.add(new Field("term_vectors", "test" + docId, ft));
+          if (rarely() == false) {
+            doc.add(new NumericDocValuesField("id", docId));
+            doc.add(new SortedSetDocValuesField("sorted_set_sort_field", new BytesRef(String.format(Locale.ROOT, "%06d", docId))));
+            doc.add(new SortedDocValuesField("sorted_binary_sort_field", new BytesRef(String.format(Locale.ROOT, "%06d", docId))));
+            doc.add(new SortedNumericDocValuesField("sorted_numeric_sort_field", docId));
+          } else {
+            doc.add(new NumericDocValuesField("alt_id", docId));
+          }
+          iw.addDocument(doc);
+          if (i > 0 && random().nextInt(5) == 0) {
+            final int id = RandomPicks.randomFrom(random(), docIds.subList(0, i));
+            iw.deleteDocuments(new Term("id", Integer.toString(id)));
+          }
+        }
+        iw.commit();
+        actualNumDocs = iw.getDocStats().numDocs;
+      }
+      Sort indexSort = RandomPicks.randomFrom(random(), Arrays.asList(
+          new Sort(new SortField("id", SortField.Type.INT),
+              new SortField("alt_id", SortField.Type.INT)),
+          new Sort(new SortedSetSortField("sorted_set_sort_field", false),
+              new SortField("alt_id", SortField.Type.INT)),
+          new Sort(new SortedNumericSortField("sorted_numeric_sort_field", SortField.Type.INT),
+              new SortField("alt_id", SortField.Type.INT)),
+          new Sort(new SortField("sorted_binary_sort_field", SortField.Type.STRING, false),
+              new SortField("alt_id", SortField.Type.INT))
+          ));
+      try (Directory sortDir = newDirectory()) {
+        try (IndexWriter writer = new IndexWriter(sortDir, newIndexWriterConfig().setIndexSort(indexSort))) {
+          try (DirectoryReader reader = DirectoryReader.open(dir)) {
+            List<CodecReader> readers = new ArrayList<>();
+            for (LeafReaderContext ctx : reader.leaves()) {
+              CodecReader wrap = SortingCodecReader.wrap(SlowCodecReaderWrapper.wrap(ctx.reader()), indexSort);
+              readers.add(wrap);
+              TermVectorsReader termVectorsReader = wrap.getTermVectorsReader();
+              TermVectorsReader clone = termVectorsReader.clone();
+              assertNotSame(termVectorsReader, clone);
+              clone.close();
+            }
+            writer.addIndexes(readers.toArray(new CodecReader[0]));
+          }
+          assumeTrue("must have at least one doc", actualNumDocs > 0);
+          try (DirectoryReader r = DirectoryReader.open(writer)) {
+            LeafReader leaf = getOnlyLeafReader(r);
+            assertEquals(actualNumDocs, leaf.maxDoc());
+            BinaryDocValues binary_dv = leaf.getBinaryDocValues("binary_dv");
+            SortedNumericDocValues sorted_numeric_dv = leaf.getSortedNumericDocValues("sorted_numeric_dv");
+            SortedSetDocValues sorted_set_dv = leaf.getSortedSetDocValues("sorted_set_dv");
+            SortedDocValues binary_sorted_dv = leaf.getSortedDocValues("binary_sorted_dv");
+            NumericDocValues ids = leaf.getNumericDocValues("id");
+            long prevValue = -1;
+            boolean usingAltIds = false;
+            for (int i = 0; i < actualNumDocs; i++) {
+              int idNext = ids.nextDoc();
+              if (idNext == DocIdSetIterator.NO_MORE_DOCS) {
+                assertFalse(usingAltIds);
+                usingAltIds = true;
+                ids = leaf.getNumericDocValues("alt_id");
+                idNext = ids.nextDoc();
+                binary_dv = leaf.getBinaryDocValues("binary_dv");
+                sorted_numeric_dv = leaf.getSortedNumericDocValues("sorted_numeric_dv");
+                sorted_set_dv = leaf.getSortedSetDocValues("sorted_set_dv");
+                binary_sorted_dv = leaf.getSortedDocValues("binary_sorted_dv");
+                prevValue = -1;
+              }
+              assertTrue(prevValue + " < " + ids.longValue(), prevValue < ids.longValue());
+              prevValue = ids.longValue();
+              assertTrue(binary_dv.advanceExact(idNext));
+              assertTrue(sorted_numeric_dv.advanceExact(idNext));
+              assertTrue(sorted_set_dv.advanceExact(idNext));
+              assertTrue(binary_sorted_dv.advanceExact(idNext));
+              assertEquals(new BytesRef(ids.longValue() + ""), binary_dv.binaryValue());
+              assertEquals(new BytesRef(ids.longValue() + ""), binary_sorted_dv.binaryValue());
+              assertEquals(new BytesRef(ids.longValue() + ""), sorted_set_dv.lookupOrd(sorted_set_dv.nextOrd()));
+              assertEquals(1, sorted_numeric_dv.docValueCount());
+              assertEquals(ids.longValue(), sorted_numeric_dv.nextValue());
+              Fields termVectors = leaf.getTermVectors(idNext);
+              assertTrue(termVectors.terms("term_vectors").iterator().seekExact(new BytesRef("test" + ids.longValue())));
+              assertEquals(Long.toString(ids.longValue()), leaf.document(idNext).get("id"));
+              IndexSearcher searcher = new IndexSearcher(r);
+              TopDocs result = searcher.search(LongPoint.newExactQuery("id", ids.longValue()), 1);
+              assertEquals(1, result.totalHits.value);
+              assertEquals(idNext, result.scoreDocs[0].doc);
+
+              result = searcher.search(new TermQuery(new Term("id", "" + ids.longValue())), 1);
+              assertEquals(1, result.totalHits.value);
+              assertEquals(idNext, result.scoreDocs[0].doc);
+            }
+            assertEquals(DocIdSetIterator.NO_MORE_DOCS, ids.nextDoc());
+          }
+        }
+      }
+    }
+  }
+}