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

[03/21] lucene-solr:branch_6x: LUCENE-6766: initial patch

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
new file mode 100644
index 0000000..45d4482
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
@@ -0,0 +1,918 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.index.Sorter.DocMap;
+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.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.TimSorter;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+
+/**
+ * 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 when it needs to merge
+ * a newly flushed (unsorted) segment.
+ *
+ * @lucene.experimental
+ */
+
+class SortingLeafReader extends FilterLeafReader {
+
+  private 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);
+    }
+
+  }
+
+  private static class SortingBinaryDocValues extends BinaryDocValues {
+
+    private final BinaryDocValues in;
+    private final Sorter.DocMap docMap;
+
+    SortingBinaryDocValues(BinaryDocValues in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public BytesRef get(int docID) {
+      return in.get(docMap.newToOld(docID));
+    }
+  }
+
+  private static class SortingNumericDocValues extends NumericDocValues {
+
+    private final NumericDocValues in;
+    private final Sorter.DocMap docMap;
+
+    public SortingNumericDocValues(final NumericDocValues in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public long get(int docID) {
+      return in.get(docMap.newToOld(docID));
+    }
+  }
+
+  private static class SortingSortedNumericDocValues extends SortedNumericDocValues {
+
+    private final SortedNumericDocValues in;
+    private final Sorter.DocMap docMap;
+
+    SortingSortedNumericDocValues(SortedNumericDocValues in, DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public int count() {
+      return in.count();
+    }
+
+    @Override
+    public void setDocument(int doc) {
+      in.setDocument(docMap.newToOld(doc));
+    }
+
+    @Override
+    public long valueAt(int index) {
+      return in.valueAt(index);
+    }
+  }
+
+  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(String fieldName, IntersectVisitor visitor) throws IOException {
+      in.intersect(fieldName,
+                   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 byte[] getMinPackedValue(String fieldName) throws IOException {
+      return in.getMinPackedValue(fieldName);
+    }
+
+    @Override
+    public byte[] getMaxPackedValue(String fieldName) throws IOException {
+      return in.getMaxPackedValue(fieldName);
+    }
+
+    @Override
+    public int getNumDimensions(String fieldName) throws IOException {
+      return in.getNumDimensions(fieldName);
+    }
+
+    @Override
+    public int getBytesPerDimension(String fieldName) throws IOException {
+      return in.getBytesPerDimension(fieldName);
+    }
+
+    @Override
+    public long size(String fieldName) {
+      return in.size(fieldName);
+    }
+
+    @Override
+    public int getDocCount(String fieldName) {
+      return in.getDocCount(fieldName);
+    }
+  }
+
+  private static class SortingSortedDocValues extends SortedDocValues {
+
+    private final SortedDocValues in;
+    private final Sorter.DocMap docMap;
+
+    SortingSortedDocValues(SortedDocValues in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public int getOrd(int docID) {
+      return in.getOrd(docMap.newToOld(docID));
+    }
+
+    @Override
+    public BytesRef lookupOrd(int ord) {
+      return in.lookupOrd(ord);
+    }
+
+    @Override
+    public int getValueCount() {
+      return in.getValueCount();
+    }
+
+    @Override
+    public BytesRef get(int docID) {
+      return in.get(docMap.newToOld(docID));
+    }
+
+    @Override
+    public int lookupTerm(BytesRef key) {
+      return in.lookupTerm(key);
+    }
+  }
+
+  private static class SortingSortedSetDocValues extends SortedSetDocValues {
+
+    private final SortedSetDocValues in;
+    private final Sorter.DocMap docMap;
+
+    SortingSortedSetDocValues(SortedSetDocValues in, Sorter.DocMap docMap) {
+      this.in = in;
+      this.docMap = docMap;
+    }
+
+    @Override
+    public long nextOrd() {
+      // nocommit
+      long v = in.nextOrd();
+      //System.out.println("  slr.sssdv.nextOrd return " + v + " this=" + this);
+      return v;
+    }
+
+    @Override
+    public void setDocument(int docID) {
+      //System.out.println("  slr.sssdv.setDocument docID=" + docID + " this=" + this);
+      in.setDocument(docMap.newToOld(docID));
+    }
+
+    @Override
+    public BytesRef lookupOrd(long ord) {
+      return in.lookupOrd(ord);
+    }
+
+    @Override
+    public long getValueCount() {
+      return in.getValueCount();
+    }
+
+    @Override
+    public long lookupTerm(BytesRef key) {
+      return in.lookupTerm(key);
+    }
+  }
+
+  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 = Arrays.copyOf(docs, newLength);
+          offsets = Arrays.copyOf(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 Fields fields() throws IOException {
+    return new SortingFields(in.fields(), in.getFieldInfos(), docMap);
+  }
+
+  @Override
+  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
+    BinaryDocValues oldDocValues = in.getBinaryDocValues(field);
+    if (oldDocValues == null) {
+      return null;
+    } else {
+      return new SortingBinaryDocValues(oldDocValues, docMap);
+    }
+  }
+
+  @Override
+  public Bits getLiveDocs() {
+    final Bits inLiveDocs = in.getLiveDocs();
+    if (inLiveDocs == null) {
+      return null;
+    } else {
+      return new SortingBits(inLiveDocs, docMap);
+    }
+  }
+
+  @Override
+  public PointValues getPointValues() {
+    final PointValues inPointValues = in.getPointValues();
+    if (inPointValues == null) {
+      return null;
+    } else {
+      // nocommit make sure this is tested
+      return new SortingPointValues(inPointValues, docMap);
+    }
+  }
+
+  @Override
+  public NumericDocValues getNormValues(String field) throws IOException {
+    final NumericDocValues norm = in.getNormValues(field);
+    if (norm == null) {
+      return null;
+    } else {
+      return new SortingNumericDocValues(norm, docMap);
+    }
+  }
+
+  @Override
+  public NumericDocValues getNumericDocValues(String field) throws IOException {
+    final NumericDocValues oldDocValues = in.getNumericDocValues(field);
+    if (oldDocValues == null) return null;
+    return new SortingNumericDocValues(oldDocValues, docMap);
+  }
+
+  @Override
+  public SortedNumericDocValues getSortedNumericDocValues(String field)
+      throws IOException {
+    final SortedNumericDocValues oldDocValues = in.getSortedNumericDocValues(field);
+    if (oldDocValues == null) {
+      return null;
+    } else {
+      return new SortingSortedNumericDocValues(oldDocValues, docMap);
+    }
+  }
+
+  @Override
+  public SortedDocValues getSortedDocValues(String field) throws IOException {
+    SortedDocValues sortedDV = in.getSortedDocValues(field);
+    if (sortedDV == null) {
+      return null;
+    } else {
+      return new SortingSortedDocValues(sortedDV, docMap);
+    }
+  }
+
+  @Override
+  public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
+    SortedSetDocValues sortedSetDV = in.getSortedSetDocValues(field);
+    if (sortedSetDV == null) {
+      return null;
+    } else {
+      return new SortingSortedSetDocValues(sortedSetDV, docMap);
+    }
+  }
+
+  @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    Bits bits = in.getDocsWithField(field);
+    if (bits == null || bits instanceof Bits.MatchAllBits || bits instanceof Bits.MatchNoBits) {
+      return bits;
+    } else {
+      return new SortingBits(bits, docMap);
+    }
+  }
+
+  @Override
+  public Fields getTermVectors(final int docID) throws IOException {
+    return in.getTermVectors(docMap.newToOld(docID));
+  }
+
+  @Override
+  public String toString() {
+    return "SortingLeafReader(" + in + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/search/Sort.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Sort.java b/lucene/core/src/java/org/apache/lucene/search/Sort.java
index 7493e9b..77585a2 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Sort.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Sort.java
@@ -147,6 +147,9 @@ public class Sort {
    *  etc.  Finally, if there is still a tie after all SortFields
    *  are checked, the internal Lucene docid is used to break it. */
   public void setSort(SortField... fields) {
+    if (fields.length == 0) {
+      throw new IllegalArgumentException("There must be at least 1 sort field");
+    }
     this.fields = fields;
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index e13e5cf..ad9dd5d 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -299,9 +299,6 @@ public class BKDWriter implements Closeable {
     final BKDReader.IntersectState state;
     final MergeState.DocMap docMap;
 
-    /** Base offset for all our docIDs */
-    final int docIDBase;
-
     /** Current doc ID */
     public int docID;
 
@@ -314,7 +311,7 @@ public class BKDWriter implements Closeable {
     /** Which leaf block we are up to */
     private int blockID;
 
-    public MergeReader(BKDReader bkd, MergeState.DocMap docMap, int docIDBase) throws IOException {
+    public MergeReader(BKDReader bkd, MergeState.DocMap docMap) throws IOException {
       this.bkd = bkd;
       state = new BKDReader.IntersectState(bkd.in.clone(),
                                            bkd.numDims,
@@ -322,7 +319,6 @@ public class BKDWriter implements Closeable {
                                            bkd.maxPointsInLeafNode,
                                            null);
       this.docMap = docMap;
-      this.docIDBase = docIDBase;
       long minFP = Long.MAX_VALUE;
       //System.out.println("MR.init " + this + " bkdreader=" + bkd + " leafBlockFPs.length=" + bkd.leafBlockFPs.length);
       for(long fp : bkd.leafBlockFPs) {
@@ -396,14 +392,14 @@ public class BKDWriter implements Closeable {
       }
 
       // Tie break by sorting smaller docIDs earlier:
-      return a.docIDBase < b.docIDBase;
+      return a.docID < b.docID;
     }
   }
 
   /** More efficient bulk-add for incoming {@link BKDReader}s.  This does a merge sort of the already
    *  sorted values and currently only works when numDims==1.  This returns -1 if all documents containing
    *  dimensional values were deleted. */
-  public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers, List<Integer> docIDBases) throws IOException {
+  public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
     if (numDims != 1) {
       throw new UnsupportedOperationException("numDims must be 1 but got " + numDims);
     }
@@ -411,8 +407,6 @@ public class BKDWriter implements Closeable {
       throw new IllegalStateException("cannot mix add and merge");
     }
 
-    //System.out.println("BKDW.merge segs=" + readers.size());
-
     // Catch user silliness:
     if (heapPointWriter == null && tempInput == null) {
       throw new IllegalStateException("already finished");
@@ -433,7 +427,7 @@ public class BKDWriter implements Closeable {
       } else {
         docMap = docMaps.get(i);
       }
-      MergeReader reader = new MergeReader(bkd, docMap, docIDBases.get(i));
+      MergeReader reader = new MergeReader(bkd, docMap);
       if (reader.next()) {
         queue.add(reader);
       }
@@ -468,7 +462,7 @@ public class BKDWriter implements Closeable {
       // System.out.println("iter reader=" + reader);
 
       // NOTE: doesn't work with subclasses (e.g. SimpleText!)
-      int docID = reader.docIDBase + reader.docID;
+      int docID = reader.docID;
       leafBlockDocIDs[leafCount] = docID;
       System.arraycopy(reader.state.scratchPackedValue, 0, leafBlockPackedValues[leafCount], 0, packedBytesLength);
       docsSeen.set(docID);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
index 73b4622..bd1e9b6 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
@@ -222,7 +222,7 @@ public class TestCodecs extends LuceneTestCase {
     final FieldInfos fieldInfos = builder.finish();
     final Directory dir = newDirectory();
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
     
     this.write(si, fieldInfos, dir, fields);
     final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
@@ -279,7 +279,7 @@ public class TestCodecs extends LuceneTestCase {
     }
 
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
     this.write(si, fieldInfos, dir, fields);
 
     if (VERBOSE) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
index 9f3339c..0dc654c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -503,7 +503,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
 
       class ReindexingOneMerge extends OneMerge {
 
-        List<LeafReader> parallelReaders;
+        final List<ParallelLeafReader> parallelReaders = new ArrayList<>();
         final long schemaGen;
 
         ReindexingOneMerge(List<SegmentCommitInfo> segments) {
@@ -519,33 +519,23 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
         }
 
         @Override
-        public List<CodecReader> getMergeReaders() throws IOException {
-          if (parallelReaders == null) {
-            parallelReaders = new ArrayList<>();
-            for (CodecReader reader : super.getMergeReaders()) {
-              parallelReaders.add(getCurrentReader((SegmentReader)reader, schemaGen));
-            }
-          }
-
-          // TODO: fix ParallelLeafReader, if this is a good use case
-          List<CodecReader> mergeReaders = new ArrayList<>();
-          for (LeafReader reader : parallelReaders) {
-            mergeReaders.add(SlowCodecReaderWrapper.wrap(reader));
+        public CodecReader wrapForMerge(CodecReader reader) throws IOException {
+          LeafReader wrapped = getCurrentReader((SegmentReader)reader, schemaGen);
+          if (wrapped instanceof ParallelLeafReader) {
+            parallelReaders.add((ParallelLeafReader) wrapped);
           }
-          return mergeReaders;
+          return SlowCodecReaderWrapper.wrap(wrapped);
         }
 
         @Override
         public void mergeFinished() throws IOException {
           Throwable th = null;
-          for(LeafReader r : parallelReaders) {
-            if (r instanceof ParallelLeafReader) {
-              try {
-                r.decRef();
-              } catch (Throwable t) {
-                if (th == null) {
-                  th = t;
-                }
+          for (ParallelLeafReader r : parallelReaders) {
+            try {
+              r.decRef();
+            } catch (Throwable t) {
+              if (th == null) {
+                th = t;
               }
             }
           }
@@ -561,10 +551,6 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
           super.setMergeInfo(info);
         }
 
-        @Override
-        public MergePolicy.DocMap getDocMap(final MergeState mergeState) {
-          return super.getDocMap(mergeState);
-        }
       }
 
       class ReindexingMergeSpecification extends MergeSpecification {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
index 803b1d9..8b24b4d 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
@@ -218,7 +218,7 @@ public class TestDoc extends LuceneTestCase {
 
     final Codec codec = Codec.getDefault();
     TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
-    final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
 
     SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(r1, r2),
                                              si, InfoStream.getDefault(), trackingDir,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
new file mode 100644
index 0000000..9251b00
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
@@ -0,0 +1,179 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+public class TestDocIDMerger extends LuceneTestCase {
+
+  private static class TestSubUnsorted extends DocIDMerger.Sub {
+    private int docID = -1;
+    final int valueStart;
+    final int maxDoc;
+
+    public TestSubUnsorted(MergeState.DocMap docMap, Bits liveDocs, int maxDoc, int valueStart) {
+      super(docMap, liveDocs);
+      this.maxDoc = maxDoc;
+      this.valueStart = valueStart;
+    }
+
+    @Override
+    public int nextDoc() {
+      docID++;
+      if (docID == maxDoc) {
+        return NO_MORE_DOCS;
+      } else {
+        return docID;
+      }
+    }
+
+    public int getValue() {
+      return valueStart + docID;
+    }
+  }
+
+  public void testNoSort() throws Exception {
+
+    int subCount = TestUtil.nextInt(random(), 1, 20);
+    List<TestSubUnsorted> subs = new ArrayList<>();
+    int valueStart = 0;
+    for(int i=0;i<subCount;i++) {
+      int maxDoc = TestUtil.nextInt(random(), 1, 1000);
+      final int docBase = valueStart;
+      subs.add(new TestSubUnsorted(new MergeState.DocMap() {
+          @Override
+          public int get(int docID) {
+            return docBase + docID;
+          }
+        }, null, maxDoc, valueStart));
+      valueStart += maxDoc;
+    }
+
+    DocIDMerger<TestSubUnsorted> merger = new DocIDMerger<>(subs, false);
+
+    int count = 0;
+    while (true) {
+      TestSubUnsorted sub = merger.next();
+      if (sub == null) {
+        break;
+      }
+      assertEquals(count, sub.mappedDocID);
+      assertEquals(count, sub.getValue());
+      count++;
+    }
+
+    assertEquals(valueStart, count);
+  }
+
+  private static class TestSubSorted extends DocIDMerger.Sub {
+    private int docID = -1;
+    final int maxDoc;
+    final int index;
+
+    public TestSubSorted(MergeState.DocMap docMap, Bits liveDocs, int maxDoc, int index) {
+      super(docMap, liveDocs);
+      this.maxDoc = maxDoc;
+      this.index = index;
+    }
+
+    @Override
+    public int nextDoc() {
+      docID++;
+      if (docID == maxDoc) {
+        return NO_MORE_DOCS;
+      } else {
+        return docID;
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "TestSubSorted(index=" + index + ", mappedDocID=" + mappedDocID+ ")";
+    }
+  }
+
+  public void testWithSort() throws Exception {
+
+    int subCount = TestUtil.nextInt(random(), 1, 20);
+    List<int[]> oldToNew = new ArrayList<>();
+    // how many docs we've written to each sub:
+    List<Integer> uptos = new ArrayList<>();
+    int totDocCount = 0;
+    for(int i=0;i<subCount;i++) {
+      int maxDoc = TestUtil.nextInt(random(), 1, 1000);
+      uptos.add(0);
+      oldToNew.add(new int[maxDoc]);
+      totDocCount += maxDoc;
+    }
+
+    List<int[]> completedSubs = new ArrayList<>();
+
+    // randomly distribute target docIDs into the segments:
+    for(int docID=0;docID<totDocCount;docID++) {
+      int sub = random().nextInt(oldToNew.size());
+      int upto = uptos.get(sub);
+      int[] subDocs = oldToNew.get(sub);
+      subDocs[upto] = docID;
+      upto++;
+      if (upto == subDocs.length) {
+        completedSubs.add(subDocs);
+        oldToNew.remove(sub);
+        uptos.remove(sub);
+      } else {
+        uptos.set(sub, upto);
+      }
+    }
+    assertEquals(0, oldToNew.size());
+
+    List<TestSubSorted> subs = new ArrayList<>();
+    for(int i=0;i<subCount;i++) {
+      final int[] docMap = completedSubs.get(i);
+      subs.add(new TestSubSorted(new MergeState.DocMap() {
+          @Override
+          public int get(int docID) {
+            return docMap[docID];
+          }
+        }, null, docMap.length, i));
+    }
+
+    // nocommit test w/ deletions too
+
+    DocIDMerger<TestSubSorted> merger = new DocIDMerger<>(subs, true);
+
+    int count = 0;
+    while (true) {
+      TestSubSorted sub = merger.next();
+      if (sub == null) {
+        break;
+      }
+      assertEquals(count, sub.mappedDocID);
+      count++;
+    }
+
+    assertEquals(totDocCount, count);
+  }
+
+  // nocommit more tests, e.g. deleted docs
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
new file mode 100644
index 0000000..2635b00
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -0,0 +1,792 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.BinaryPoint;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+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.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TermStatistics;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+// nocommit test tie break
+// nocommit test multiple sorts
+// nocommit test update dvs
+
+// nocommit test EarlyTerminatingCollector
+
+public class TestIndexSorting extends LuceneTestCase {
+
+  public void testSortOnMerge(boolean withDeletes) throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
+    Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
+    iwc.setIndexSort(indexSort);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    final int numDocs = atLeast(200);
+    final FixedBitSet deleted = new FixedBitSet(numDocs);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new NumericDocValuesField("foo", random().nextInt(20)));
+      doc.add(new StringField("id", Integer.toString(i), Store.YES));
+      doc.add(new NumericDocValuesField("id", i));
+      w.addDocument(doc);
+      if (random().nextInt(5) == 0) {
+        w.getReader().close();
+      } else if (random().nextInt(30) == 0) {
+        w.forceMerge(2);
+      } else if (random().nextInt(4) == 0) {
+        final int id = TestUtil.nextInt(random(), 0, i);
+        deleted.set(id);
+        w.deleteDocuments(new Term("id", Integer.toString(id)));
+      }
+    }
+
+    // Check that segments are sorted
+    DirectoryReader reader = w.getReader();
+    for (LeafReaderContext ctx : reader.leaves()) {
+      final SegmentReader leaf = (SegmentReader) ctx.reader();
+      SegmentInfo info = leaf.getSegmentInfo().info;
+      switch (info.getDiagnostics().get(IndexWriter.SOURCE)) {
+        case IndexWriter.SOURCE_FLUSH:
+          assertNull(info.getIndexSort());
+          break;
+        case IndexWriter.SOURCE_MERGE:
+          assertEquals(indexSort, info.getIndexSort());
+          final NumericDocValues values = leaf.getNumericDocValues("foo");
+          long previous = Long.MIN_VALUE;
+          for (int i = 0; i < leaf.maxDoc(); ++i) {
+            final long value = values.get(i);
+            assertTrue(value >= previous);
+            previous = value;
+          }
+          break;
+        default:
+          fail();
+      }
+    }
+
+    // Now check that the index is consistent
+    IndexSearcher searcher = newSearcher(reader);
+    for (int i = 0; i < numDocs; ++i) {
+      TermQuery termQuery = new TermQuery(new Term("id", Integer.toString(i)));
+      final TopDocs topDocs = searcher.search(termQuery, 1);
+      if (deleted.get(i)) {
+        assertEquals(0, topDocs.totalHits);
+      } else {
+        assertEquals(1, topDocs.totalHits);
+        assertEquals(i, MultiDocValues.getNumericValues(reader, "id").get(topDocs.scoreDocs[0].doc));
+        Document document = reader.document(topDocs.scoreDocs[0].doc);
+        assertEquals(Integer.toString(i), document.get("id"));
+      }
+    }
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testSortOnMerge() throws IOException {
+    testSortOnMerge(false);
+  }
+
+  public void testSortOnMergeWithDeletes() throws IOException {
+    testSortOnMerge(true);
+  }
+
+  static class UpdateRunnable implements Runnable {
+
+    private final int numDocs;
+    private final Random random;
+    private final AtomicInteger updateCount;
+    private final IndexWriter w;
+    private final Map<Integer, Long> values;
+    private final CountDownLatch latch;
+
+    UpdateRunnable(int numDocs, Random random, CountDownLatch latch, AtomicInteger updateCount, IndexWriter w, Map<Integer, Long> values) {
+      this.numDocs = numDocs;
+      this.random = random;
+      this.latch = latch;
+      this.updateCount = updateCount;
+      this.w = w;
+      this.values = values;
+    }
+
+    @Override
+    public void run() {
+      try {
+        latch.await();
+        while (updateCount.decrementAndGet() >= 0) {
+          final int id = random.nextInt(numDocs);
+          final long value = random.nextInt(20);
+          Document doc = new Document();
+          doc.add(new StringField("id", Integer.toString(id), Store.NO));
+          doc.add(new NumericDocValuesField("foo", value));
+
+          synchronized (values) {
+            w.updateDocument(new Term("id", Integer.toString(id)), doc);
+            values.put(id, value);
+          }
+
+          switch (random.nextInt(10)) {
+            case 0:
+            case 1:
+              // reopen
+              DirectoryReader.open(w).close();
+              break;
+            case 2:
+              w.forceMerge(3);
+              break;
+          }
+        }
+      } catch (IOException | InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+  }
+
+  // There is tricky logic to resolve deletes that happened while merging
+  public void testConcurrentUpdates() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
+    Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
+    iwc.setIndexSort(indexSort);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    Map<Integer, Long> values = new HashMap<>();
+
+    final int numDocs = atLeast(100);
+    Thread[] threads = new Thread[2];
+    final AtomicInteger updateCount = new AtomicInteger(atLeast(1000));
+    final CountDownLatch latch = new CountDownLatch(1);
+    for (int i = 0; i < threads.length; ++i) {
+      Random r = new Random(random().nextLong());
+      threads[i] = new Thread(new UpdateRunnable(numDocs, r, latch, updateCount, w, values));
+    }
+    for (Thread thread : threads) {
+      thread.start();
+    }
+    latch.countDown();
+    for (Thread thread : threads) {
+      thread.join();
+    }
+    w.forceMerge(1);
+    DirectoryReader reader = DirectoryReader.open(w);
+    IndexSearcher searcher = newSearcher(reader);
+    for (int i = 0; i < numDocs; ++i) {
+      final TopDocs topDocs = searcher.search(new TermQuery(new Term("id", Integer.toString(i))), 1);
+      if (values.containsKey(i) == false) {
+        assertEquals(0, topDocs.totalHits);
+      } else {
+        assertEquals(1, topDocs.totalHits);
+        assertEquals(values.get(i).longValue(), MultiDocValues.getNumericValues(reader, "foo").get(topDocs.scoreDocs[0].doc));
+      }
+    }
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  static class DVUpdateRunnable implements Runnable {
+
+    private final int numDocs;
+    private final Random random;
+    private final AtomicInteger updateCount;
+    private final IndexWriter w;
+    private final Map<Integer, Long> values;
+    private final CountDownLatch latch;
+
+    DVUpdateRunnable(int numDocs, Random random, CountDownLatch latch, AtomicInteger updateCount, IndexWriter w, Map<Integer, Long> values) {
+      this.numDocs = numDocs;
+      this.random = random;
+      this.latch = latch;
+      this.updateCount = updateCount;
+      this.w = w;
+      this.values = values;
+    }
+
+    @Override
+    public void run() {
+      try {
+        latch.await();
+        while (updateCount.decrementAndGet() >= 0) {
+          final int id = random.nextInt(numDocs);
+          final long value = random.nextInt(20);
+
+          synchronized (values) {
+            w.updateDocValues(new Term("id", Integer.toString(id)), new NumericDocValuesField("foo", value));
+            values.put(id, value);
+          }
+
+          switch (random.nextInt(10)) {
+            case 0:
+            case 1:
+              // reopen
+              DirectoryReader.open(w).close();
+              break;
+            case 2:
+              w.forceMerge(3);
+              break;
+          }
+        }
+      } catch (IOException | InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+  }
+
+  // There is tricky logic to resolve dv updates that happened while merging
+  public void testConcurrentDVUpdates() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
+    Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
+    iwc.setIndexSort(indexSort);
+    IndexWriter w = new IndexWriter(dir, iwc);
+    Map<Integer, Long> values = new HashMap<>();
+
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("id", Integer.toString(i), Store.NO));
+      doc.add(new NumericDocValuesField("foo", -1));
+      values.put(i, -1L);
+    }
+    Thread[] threads = new Thread[2];
+    final AtomicInteger updateCount = new AtomicInteger(atLeast(1000));
+    final CountDownLatch latch = new CountDownLatch(1);
+    for (int i = 0; i < threads.length; ++i) {
+      Random r = new Random(random().nextLong());
+      threads[i] = new Thread(new UpdateRunnable(numDocs, r, latch, updateCount, w, values));
+    }
+    for (Thread thread : threads) {
+      thread.start();
+    }
+    latch.countDown();
+    for (Thread thread : threads) {
+      thread.join();
+    }
+    w.forceMerge(1);
+    DirectoryReader reader = DirectoryReader.open(w);
+    IndexSearcher searcher = newSearcher(reader);
+    for (int i = 0; i < numDocs; ++i) {
+      final TopDocs topDocs = searcher.search(new TermQuery(new Term("id", Integer.toString(i))), 1);
+      assertEquals(1, topDocs.totalHits);
+      assertEquals(values.get(i).longValue(), MultiDocValues.getNumericValues(reader, "foo").get(topDocs.scoreDocs[0].doc));
+    }
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testAddIndexes(boolean withDeletes) throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("id", Integer.toString(i), Store.NO));
+      doc.add(new NumericDocValuesField("foo", random().nextInt(20)));
+      w.addDocument(doc);
+    }
+    if (withDeletes) {
+      for (int i = random().nextInt(5); i < numDocs; i += TestUtil.nextInt(random(), 1, 5)) {
+        w.deleteDocuments(new Term("id", Integer.toString(i)));
+      }
+    }
+    final IndexReader reader = w.getReader();
+
+    Directory dir2 = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
+    Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
+    iwc.setIndexSort(indexSort);
+    IndexWriter w2 = new IndexWriter(dir2, iwc);
+
+    CodecReader[] codecReaders = new CodecReader[reader.leaves().size()];
+    for (int i = 0; i < codecReaders.length; ++i) {
+      codecReaders[i] = (CodecReader) reader.leaves().get(i).reader();
+    }
+    w2.addIndexes(codecReaders);
+    final IndexReader reader2 = w2.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    final IndexSearcher searcher2 = newSearcher(reader2);
+    for (int i = 0; i < numDocs; ++i) {
+      Query query = new TermQuery(new Term("id", Integer.toString(i)));
+      final TopDocs topDocs = searcher.search(query, 1);
+      final TopDocs topDocs2 = searcher2.search(query, 1);
+      assertEquals(topDocs.totalHits, topDocs2.totalHits);
+      if (topDocs.totalHits == 1) {
+        assertEquals(
+            MultiDocValues.getNumericValues(reader, "foo").get(topDocs.scoreDocs[0].doc),
+            MultiDocValues.getNumericValues(reader2, "foo").get(topDocs2.scoreDocs[0].doc));
+      }
+    }
+
+    IOUtils.close(reader, reader2, w, w2, dir, dir2);
+  }
+
+  public void testAddIndexes() throws Exception {
+    testAddIndexes(false);
+  }
+
+  public void testAddIndexesWithDeletions() throws Exception {
+    testAddIndexes(true);
+  }
+
+  public void testBadSort() throws Exception {
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      iwc.setIndexSort(Sort.RELEVANCE);
+    });
+    assertEquals("invalid SortField type: must be one of [STRING, INT, FLOAT, LONG, DOUBLE, BYTES] but got: <score>", expected.getMessage());
+  }
+
+  // you can't change the index sort on an existing index:
+  public void testIllegalChangeSort() throws Exception {
+    final Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
+    iwc.setIndexSort(new Sort(new SortField("foo", SortField.Type.LONG)));
+    IndexWriter w = new IndexWriter(dir, iwc);
+    w.addDocument(new Document());
+    DirectoryReader.open(w).close();
+    w.addDocument(new Document());
+    w.forceMerge(1);
+    w.close();
+
+    final IndexWriterConfig iwc2 = new IndexWriterConfig(new MockAnalyzer(random()));
+    iwc2.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
+    iwc2.setIndexSort(new Sort(new SortField("bar", SortField.Type.LONG)));
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+        new IndexWriter(dir, iwc2);
+    });
+    assertEquals("cannot change previous indexSort=<long: \"foo\"> (from segment=_2(7.0.0):c2:[indexSort=<long: \"foo\">]) to new indexSort=<long: \"bar\">", expected.getMessage());
+    dir.close();
+  }
+
+  static final class NormsSimilarity extends Similarity {
+    
+    private final Similarity in;
+    
+    public NormsSimilarity(Similarity in) {
+      this.in = in;
+    }
+    
+    @Override
+    public long computeNorm(FieldInvertState state) {
+      if (state.getName().equals(NORMS_FIELD)) {
+        return Float.floatToIntBits(state.getBoost());
+      } else {
+        return in.computeNorm(state);
+      }
+    }
+    
+    @Override
+    public SimWeight computeWeight(CollectionStatistics collectionStats, TermStatistics... termStats) {
+      return in.computeWeight(collectionStats, termStats);
+    }
+    
+    @Override
+    public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
+      return in.simScorer(weight, context);
+    }
+    
+  }
+  
+  static final class PositionsTokenStream extends TokenStream {
+    
+    private final CharTermAttribute term;
+    private final PayloadAttribute payload;
+    private final OffsetAttribute offset;
+    
+    private int pos, off;
+    
+    public PositionsTokenStream() {
+      term = addAttribute(CharTermAttribute.class);
+      payload = addAttribute(PayloadAttribute.class);
+      offset = addAttribute(OffsetAttribute.class);
+    }
+    
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (pos == 0) {
+        return false;
+      }
+      
+      clearAttributes();
+      term.append(DOC_POSITIONS_TERM);
+      payload.setPayload(new BytesRef(Integer.toString(pos)));
+      offset.setOffset(off, off);
+      --pos;
+      ++off;
+      return true;
+    }
+    
+    void setId(int id) {
+      pos = id / 10 + 1;
+      off = 0;
+    }
+  }
+
+  private static Directory dir;
+  private static IndexReader sortedReader;
+
+  private static final FieldType TERM_VECTORS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
+  static {
+    TERM_VECTORS_TYPE.setStoreTermVectors(true);
+    TERM_VECTORS_TYPE.freeze();
+  }
+  
+  private static final FieldType POSITIONS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
+  static {
+    POSITIONS_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    POSITIONS_TYPE.freeze();
+  }
+
+  private static final String ID_FIELD = "id";
+  private static final String DOCS_ENUM_FIELD = "docs";
+  private static final String DOCS_ENUM_TERM = "$all$";
+  private static final String DOC_POSITIONS_FIELD = "positions";
+  private static final String DOC_POSITIONS_TERM = "$all$";
+  private static final String NUMERIC_DV_FIELD = "numeric";
+  private static final String SORTED_NUMERIC_DV_FIELD = "sorted_numeric";
+  private static final String NORMS_FIELD = "norm";
+  private static final String BINARY_DV_FIELD = "binary";
+  private static final String SORTED_DV_FIELD = "sorted";
+  private static final String SORTED_SET_DV_FIELD = "sorted_set";
+  private static final String TERM_VECTORS_FIELD = "term_vectors";
+  private static final String DIMENSIONAL_FIELD = "numeric1d";
+
+  private static Document doc(final int id, PositionsTokenStream positions) {
+    final Document doc = new Document();
+    doc.add(new StringField(ID_FIELD, Integer.toString(id), Store.YES));
+    doc.add(new StringField(DOCS_ENUM_FIELD, DOCS_ENUM_TERM, Store.NO));
+    positions.setId(id);
+    doc.add(new Field(DOC_POSITIONS_FIELD, positions, POSITIONS_TYPE));
+    doc.add(new NumericDocValuesField(NUMERIC_DV_FIELD, id));
+    TextField norms = new TextField(NORMS_FIELD, Integer.toString(id), Store.NO);
+    norms.setBoost(Float.intBitsToFloat(id));
+    doc.add(norms);
+    doc.add(new BinaryDocValuesField(BINARY_DV_FIELD, new BytesRef(Integer.toString(id))));
+    doc.add(new SortedDocValuesField(SORTED_DV_FIELD, new BytesRef(Integer.toString(id))));
+    doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id))));
+    doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id + 1))));
+    doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id));
+    doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id + 1));
+    doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
+    byte[] bytes = new byte[4];
+    NumericUtils.intToSortableBytes(id, bytes, 0);
+    doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
+    return doc;
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (sortedReader != null) {
+      sortedReader.close();
+      sortedReader = null;
+    }
+    if (dir != null) {
+      dir.close();
+      dir = null;
+    }
+  }
+
+  @BeforeClass
+  public static void createIndex() throws Exception {
+    dir = newFSDirectory(createTempDir());
+    int numDocs = atLeast(100);
+
+    List<Integer> ids = new ArrayList<>();
+    for (int i = 0; i < numDocs; i++) {
+      ids.add(Integer.valueOf(i * 10));
+    }
+    // shuffle them for indexing
+    Collections.shuffle(ids, random());
+    if (VERBOSE) {
+      System.out.println("Shuffled IDs for indexing: " + Arrays.toString(ids.toArray()));
+    }
+    
+    PositionsTokenStream positions = new PositionsTokenStream();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    // nocommit:
+    conf.setCodec(new SimpleTextCodec());
+    conf.setMaxBufferedDocs(4); // create some segments
+    conf.setSimilarity(new NormsSimilarity(conf.getSimilarity())); // for testing norms field
+    // nocommit
+    conf.setMergeScheduler(new SerialMergeScheduler());
+    // sort the index by id (as integer, in NUMERIC_DV_FIELD)
+    conf.setIndexSort(new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.INT)));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
+    writer.setDoRandomForceMerge(false);
+    for (int id : ids) {
+      writer.addDocument(doc(id, positions));
+    }
+    // delete some documents
+    writer.commit();
+    // nocommit need thread safety test too
+    for (Integer id : ids) {
+      if (random().nextDouble() < 0.2) {
+        if (VERBOSE) {
+          System.out.println("delete doc_id " + id);
+        }
+        writer.deleteDocuments(new Term(ID_FIELD, id.toString()));
+      }
+    }
+    
+    sortedReader = writer.getReader();
+    writer.close();
+    
+    TestUtil.checkReader(sortedReader);
+  }
+
+  // nocommit just do assertReaderEquals, don't use @BeforeClass, etc.?
+
+  public void testBinaryDocValuesField() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      BinaryDocValues dv = reader.getBinaryDocValues(BINARY_DV_FIELD);
+      boolean isSorted = reader.getIndexSort() != null;
+      int lastID = Integer.MIN_VALUE;
+      for (int docID = 0; docID < reader.maxDoc(); docID++) {
+        BytesRef bytes = dv.get(docID);
+        String idString = reader.document(docID).get(ID_FIELD);
+        assertEquals("incorrect binary DocValues for doc " + docID, idString, bytes.utf8ToString());
+        if (isSorted) {
+          int id = Integer.parseInt(idString);
+          assertTrue("lastID=" + lastID + " vs id=" + id, lastID < id);
+          lastID = id;
+        }
+      }
+    }
+  }
+
+  public void testPostings() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      TermsEnum termsEnum = reader.terms(DOC_POSITIONS_FIELD).iterator();
+      assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
+      PostingsEnum sortedPositions = termsEnum.postings(null, PostingsEnum.ALL);
+      int doc;
+      boolean isSorted = reader.getIndexSort() != null;
+    
+      // test nextDoc()
+      while ((doc = sortedPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+        int freq = sortedPositions.freq();
+        int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
+        assertEquals("incorrect freq for doc=" + doc, id / 10 + 1, freq);
+        for (int i = 0; i < freq; i++) {
+          assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
+          assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
+          assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
+          assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
+        }
+      }
+    
+      // test advance()
+      final PostingsEnum reuse = sortedPositions;
+      sortedPositions = termsEnum.postings(reuse, PostingsEnum.ALL);
+
+      doc = 0;
+      while ((doc = sortedPositions.advance(doc + TestUtil.nextInt(random(), 1, 5))) != DocIdSetIterator.NO_MORE_DOCS) {
+        int freq = sortedPositions.freq();
+        int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
+        assertEquals("incorrect freq for doc=" + doc, id / 10 + 1, freq);
+        for (int i = 0; i < freq; i++) {
+          assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
+          assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
+          assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
+          assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
+        }
+      }
+    }
+  }
+
+  public void testDocsAreSortedByID() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      if (reader.getIndexSort() != null) {
+        int maxDoc = reader.maxDoc();
+        int lastID = Integer.MIN_VALUE;
+        for(int doc=0;doc<maxDoc;doc++) {
+          int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
+          assertTrue(id > lastID);
+          lastID = id;
+        }
+      }
+    }
+  }
+
+  public void testNormValues() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      NumericDocValues dv = reader.getNormValues(NORMS_FIELD);
+      int maxDoc = reader.maxDoc();
+      boolean isSorted = reader.getIndexSort() != null;
+      for (int doc = 0; doc < maxDoc; doc++) {
+        int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
+        assertEquals("incorrect norm value for doc " + doc, id, dv.get(doc));
+      }
+    }
+  }
+  
+  public void testNumericDocValuesField() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      NumericDocValues dv = reader.getNumericDocValues(NUMERIC_DV_FIELD);
+      int maxDoc = reader.maxDoc();
+      for (int doc = 0; doc < maxDoc; doc++) {
+        int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
+        assertEquals("incorrect numeric DocValues for doc " + doc, id, dv.get(doc));
+      }
+    }
+  }
+  
+  public void testSortedDocValuesField() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      SortedDocValues dv = reader.getSortedDocValues(SORTED_DV_FIELD);
+      int maxDoc = reader.maxDoc();
+      for (int doc = 0; doc < maxDoc; doc++) {
+        final BytesRef bytes = dv.get(doc);
+        String id = reader.document(doc).get(ID_FIELD);
+        assertEquals("incorrect sorted DocValues for doc " + doc, id, bytes.utf8ToString());
+      }
+    }
+  }
+  
+  public void testSortedSetDocValuesField() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      SortedSetDocValues dv = reader.getSortedSetDocValues(SORTED_SET_DV_FIELD);
+      int maxDoc = reader.maxDoc();
+      for (int doc = 0; doc < maxDoc; doc++) {
+        dv.setDocument(doc);
+        BytesRef bytes = dv.lookupOrd(dv.nextOrd());
+        String id = reader.document(doc).get(ID_FIELD);
+        assertEquals("incorrect sorted-set DocValues for doc " + doc, id, bytes.utf8ToString());
+        bytes = dv.lookupOrd(dv.nextOrd());
+        assertEquals("incorrect sorted-set DocValues for doc " + doc, Integer.valueOf(Integer.parseInt(id) + 1).toString(), bytes.utf8ToString());
+        assertEquals(SortedSetDocValues.NO_MORE_ORDS, dv.nextOrd());
+      }
+    }
+  }
+
+  public void testSortedNumericDocValuesField() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      SortedNumericDocValues dv = reader.getSortedNumericDocValues(SORTED_NUMERIC_DV_FIELD);
+      int maxDoc = reader.maxDoc();
+      for (int doc = 0; doc < maxDoc; doc++) {
+        dv.setDocument(doc);
+        assertEquals(2, dv.count());
+        int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
+        assertEquals("incorrect sorted-numeric DocValues for doc " + doc, id, dv.valueAt(0));
+        assertEquals("incorrect sorted-numeric DocValues for doc " + doc, id + 1, dv.valueAt(1));
+      }
+    }
+  }
+  
+  public void testTermVectors() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      LeafReader reader = ctx.reader();
+      int maxDoc = reader.maxDoc();
+      for (int doc = 0; doc < maxDoc; doc++) {
+        Terms terms = reader.getTermVector(doc, TERM_VECTORS_FIELD);
+        assertNotNull("term vectors not found for doc " + doc + " field [" + TERM_VECTORS_FIELD + "]", terms);
+        String id = reader.document(doc).get(ID_FIELD);
+        assertEquals("incorrect term vector for doc " + doc, id, terms.iterator().next().utf8ToString());
+      }
+    }
+  }
+
+  public void testPoints() throws Exception {
+    for(LeafReaderContext ctx : sortedReader.leaves()) {
+      final LeafReader reader = ctx.reader();
+      PointValues values = reader.getPointValues();
+      values.intersect(DIMENSIONAL_FIELD,
+                       new IntersectVisitor() {
+                         @Override
+                         public void visit(int docID) {
+                           throw new IllegalStateException();
+                         }
+
+                         @Override
+                         public void visit(int docID, byte[] packedValues) throws IOException {
+                           int id = Integer.parseInt(reader.document(docID).get(ID_FIELD));
+                           assertEquals(id, NumericUtils.sortableBytesToInt(packedValues, 0));
+                         }
+
+                         @Override
+                         public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                           return Relation.CELL_CROSSES_QUERY;
+                         }
+                       });
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index 2c3543e..fb3e07e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -69,6 +69,8 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BaseDirectoryWrapper;
@@ -2759,5 +2761,6 @@ public class TestIndexWriter extends LuceneTestCase {
     w.close();
     dir.close();
   }
+
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
index 733f75e..fb7b890 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
@@ -51,7 +51,7 @@ public class TestSegmentInfos extends LuceneTestCase {
 
     SegmentInfos sis = new SegmentInfos();
     SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_5_0_0, "_0", 1, false, Codec.getDefault(), 
-                                       Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
+                                       Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
@@ -72,15 +72,25 @@ public class TestSegmentInfos extends LuceneTestCase {
     Codec codec = Codec.getDefault();
 
     SegmentInfos sis = new SegmentInfos();
+<<<<<<< HEAD
     SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_5_0_0, "_0", 1, false, Codec.getDefault(), 
                                        Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
+=======
+    SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_0", 1, false, Codec.getDefault(), 
+                                       Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
+>>>>>>> 54fa7df... LUCENE-6766: initial patch
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
     sis.add(commitInfo);
 
+<<<<<<< HEAD
     info = new SegmentInfo(dir, Version.LUCENE_5_1_0, "_1", 1, false, Codec.getDefault(), 
                                        Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
+=======
+    info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_1", 1, false, Codec.getDefault(), 
+                           Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
+>>>>>>> 54fa7df... LUCENE-6766: initial patch
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
index e1075ab..1ef37c0 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
@@ -35,6 +35,7 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util.packed.PackedLongValues;
 
 public class TestSegmentMerger extends LuceneTestCase {
   //The variables for the new merged segment
@@ -83,7 +84,7 @@ public class TestSegmentMerger extends LuceneTestCase {
 
   public void testMerge() throws IOException {
     final Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
 
     SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(reader1, reader2),
                                              si, InfoStream.getDefault(), mergedDir,
@@ -144,22 +145,9 @@ public class TestSegmentMerger extends LuceneTestCase {
     mergedReader.close();
   }
 
-  private static boolean equals(MergeState.DocMap map1, MergeState.DocMap map2) {
-    if (map1.maxDoc() != map2.maxDoc()) {
-      return false;
-    }
-    for (int i = 0; i < map1.maxDoc(); ++i) {
-      if (map1.get(i) != map2.get(i)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
   public void testBuildDocMap() {
     final int maxDoc = TestUtil.nextInt(random(), 1, 128);
     final int numDocs = TestUtil.nextInt(random(), 0, maxDoc);
-    final int numDeletedDocs = maxDoc - numDocs;
     final FixedBitSet liveDocs = new FixedBitSet(maxDoc);
     for (int i = 0; i < numDocs; ++i) {
       while (true) {
@@ -171,15 +159,11 @@ public class TestSegmentMerger extends LuceneTestCase {
       }
     }
 
-    final MergeState.DocMap docMap = MergeState.DocMap.build(maxDoc, liveDocs);
+    final PackedLongValues docMap = MergeState.removeDeletes(maxDoc, liveDocs);
 
-    assertEquals(maxDoc, docMap.maxDoc());
-    assertEquals(numDocs, docMap.numDocs());
-    assertEquals(numDeletedDocs, docMap.numDeletedDocs());
     // assert the mapping is compact
     for (int i = 0, del = 0; i < maxDoc; ++i) {
-      if (!liveDocs.get(i)) {
-        assertEquals(-1, docMap.get(i));
+      if (liveDocs.get(i) == false) {
         ++del;
       } else {
         assertEquals(i - del, docMap.get(i));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
index 38b3fb5..b1a8f8d 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
@@ -25,6 +25,7 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.store.CorruptingIndexOutput;
@@ -554,7 +555,7 @@ public class TestBKD extends LuceneTestCase {
     }
 
     List<Long> toMerge = null;
-    List<Integer> docIDBases = null;
+    List<MergeState.DocMap> docMaps = null;
     int seg = 0;
 
     BKDWriter w = new BKDWriter(numValues, dir, "_" + seg, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
@@ -601,9 +602,15 @@ public class TestBKD extends LuceneTestCase {
         if (useMerge && segCount == valuesInThisSeg) {
           if (toMerge == null) {
             toMerge = new ArrayList<>();
-            docIDBases = new ArrayList<>();
+            docMaps = new ArrayList<>();
           }
-          docIDBases.add(lastDocIDBase);
+          final int curDocIDBase = lastDocIDBase;
+          docMaps.add(new MergeState.DocMap() {
+              @Override
+              public int get(int docID) {
+                return curDocIDBase + docID;
+              }
+            });
           toMerge.add(w.finish(out));
           valuesInThisSeg = TestUtil.nextInt(random(), numValues/10, numValues/2);
           segCount = 0;
@@ -620,8 +627,14 @@ public class TestBKD extends LuceneTestCase {
 
       if (toMerge != null) {
         if (segCount > 0) {
-          docIDBases.add(lastDocIDBase);
           toMerge.add(w.finish(out));
+          final int curDocIDBase = lastDocIDBase;
+          docMaps.add(new MergeState.DocMap() {
+              @Override
+              public int get(int docID) {
+                return curDocIDBase + docID;
+              }
+            });
         }
         out.close();
         in = dir.openInput("bkd", IOContext.DEFAULT);
@@ -633,7 +646,7 @@ public class TestBKD extends LuceneTestCase {
           readers.add(new BKDReader(in));
         }
         out = dir.createOutput("bkd2", IOContext.DEFAULT);
-        indexFP = w.merge(out, null, readers, docIDBases);
+        indexFP = w.merge(out, docMaps, readers);
         out.close();
         in.close();
         in = dir.openInput("bkd2", IOContext.DEFAULT);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
index c672ed0..368c285 100644
--- a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
+++ b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
@@ -140,7 +140,7 @@ public class IndexSplitter {
       SegmentInfo info = infoPerCommit.info;
       // Same info just changing the dir:
       SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.maxDoc(),
-                                            info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>());
+                                            info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>(), null);
       destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
           infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
           infoPerCommit.getDocValuesGen()));