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:18 UTC

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java b/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java
deleted file mode 100644
index dba5c91..0000000
--- a/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java
+++ /dev/null
@@ -1,259 +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 org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.NormsProducer;
-import org.apache.lucene.codecs.StoredFieldsReader;
-import org.apache.lucene.codecs.TermVectorsReader;
-import org.apache.lucene.util.Bits;
-
-/** this is a hack to make SortingMP fast! */
-class MergeReaderWrapper extends LeafReader {
-  final SegmentReader in;
-  final FieldsProducer fields;
-  final NormsProducer norms;
-  final DocValuesProducer docValues;
-  final StoredFieldsReader store;
-  final TermVectorsReader vectors;
-  
-  MergeReaderWrapper(SegmentReader in) throws IOException {
-    this.in = in;
-    
-    FieldsProducer fields = in.getPostingsReader();
-    if (fields != null) {
-      fields = fields.getMergeInstance();
-    }
-    this.fields = fields;
-    
-    NormsProducer norms = in.getNormsReader();
-    if (norms != null) {
-      norms = norms.getMergeInstance();
-    }
-    this.norms = norms;
-    
-    DocValuesProducer docValues = in.getDocValuesReader();
-    if (docValues != null) {
-      docValues = docValues.getMergeInstance();
-    }
-    this.docValues = docValues;
-    
-    StoredFieldsReader store = in.getFieldsReader();
-    if (store != null) {
-      store = store.getMergeInstance();
-    }
-    this.store = store;
-    
-    TermVectorsReader vectors = in.getTermVectorsReader();
-    if (vectors != null) {
-      vectors = vectors.getMergeInstance();
-    }
-    this.vectors = vectors;
-  }
-
-  @Override
-  public void addCoreClosedListener(CoreClosedListener listener) {
-    in.addCoreClosedListener(listener);
-  }
-
-  @Override
-  public void removeCoreClosedListener(CoreClosedListener listener) {
-    in.removeCoreClosedListener(listener);
-  }
-
-  @Override
-  public Fields fields() throws IOException {
-    return fields;
-  }
-
-  @Override
-  public NumericDocValues getNumericDocValues(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.NUMERIC) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    return docValues.getNumeric(fi);
-  }
-
-  @Override
-  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.BINARY) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    return docValues.getBinary(fi);
-  }
-
-  @Override
-  public SortedDocValues getSortedDocValues(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.SORTED) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    return docValues.getSorted(fi);
-  }
-
-  @Override
-  public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.SORTED_NUMERIC) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    return docValues.getSortedNumeric(fi);
-  }
-
-  @Override
-  public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() != DocValuesType.SORTED_SET) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    return docValues.getSortedSet(fi);
-  }
-
-  @Override
-  public Bits getDocsWithField(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() == DocValuesType.NONE) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    return docValues.getDocsWithField(fi);
-  }
-
-  @Override
-  public NumericDocValues getNormValues(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null || !fi.hasNorms()) {
-      // Field does not exist or does not index norms
-      return null;
-    }
-    return norms.getNorms(fi);
-  }
-
-  @Override
-  public FieldInfos getFieldInfos() {
-    return in.getFieldInfos();
-  }
-
-  @Override
-  public Bits getLiveDocs() {
-    return in.getLiveDocs();
-  }
-
-  @Override
-  public void checkIntegrity() throws IOException {
-    in.checkIntegrity();
-  }
-
-  @Override
-  public Fields getTermVectors(int docID) throws IOException {
-    ensureOpen();
-    checkBounds(docID);
-    if (vectors == null) {
-      return null;
-    }
-    return vectors.get(docID);
-  }
-
-  @Override
-  public PointValues getPointValues() {
-    return in.getPointValues();
-  }
-
-  @Override
-  public int numDocs() {
-    return in.numDocs();
-  }
-
-  @Override
-  public int maxDoc() {
-    return in.maxDoc();
-  }
-
-  @Override
-  public void document(int docID, StoredFieldVisitor visitor) throws IOException {
-    ensureOpen();
-    checkBounds(docID);
-    store.visitDocument(docID, visitor);
-  }
-
-  @Override
-  protected void doClose() throws IOException {
-    in.close();
-  }
-
-  @Override
-  public Object getCoreCacheKey() {
-    return in.getCoreCacheKey();
-  }
-
-  @Override
-  public Object getCombinedCoreAndDeletesKey() {
-    return in.getCombinedCoreAndDeletesKey();
-  }
-  
-  private void checkBounds(int docID) {
-    if (docID < 0 || docID >= maxDoc()) {       
-      throw new IndexOutOfBoundsException("docID must be >= 0 and < maxDoc=" + maxDoc() + " (got docID=" + docID + ")");
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "MergeReaderWrapper(" + in + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java b/lucene/misc/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
index 299de90..8762d73 100644
--- a/lucene/misc/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
+++ b/lucene/misc/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
 import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
+import org.apache.lucene.search.Sort;
 import org.apache.lucene.util.Bits;
 
 /**
@@ -70,6 +71,11 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
     if (getFieldInfos().hasPointValues()) {
       throw new IllegalArgumentException("cannot wrap points");
     }
+    for(LeafReaderContext context : reader.leaves()) {
+      if (context.reader().getIndexSort() != null) {
+        throw new IllegalArgumentException("cannot use index sort");
+      }
+    }
     fields = MultiFields.getFields(in);
     in.registerParentReader(this);
     this.merging = merging;
@@ -275,4 +281,9 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
       ctx.reader().checkIntegrity();
     }
   }
+
+  @Override
+  public Sort getIndexSort() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/java/org/apache/lucene/index/Sorter.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/index/Sorter.java b/lucene/misc/src/java/org/apache/lucene/index/Sorter.java
deleted file mode 100644
index 7e4e475..0000000
--- a/lucene/misc/src/java/org/apache/lucene/index/Sorter.java
+++ /dev/null
@@ -1,287 +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.Comparator;
-
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.LeafFieldComparator;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.util.TimSorter;
-import org.apache.lucene.util.packed.PackedInts;
-import org.apache.lucene.util.packed.PackedLongValues;
-
-/**
- * Sorts documents of a given index by returning a permutation on the document
- * IDs.
- * @lucene.experimental
- */
-final class Sorter {
-  final Sort sort;
-  
-  /** Creates a new Sorter to sort the index with {@code sort} */
-  Sorter(Sort sort) {
-    if (sort.needsScores()) {
-      throw new IllegalArgumentException("Cannot sort an index with a Sort that refers to the relevance score");
-    }
-    this.sort = sort;
-  }
-
-  /**
-   * A permutation of doc IDs. For every document ID between <tt>0</tt> and
-   * {@link IndexReader#maxDoc()}, <code>oldToNew(newToOld(docID))</code> must
-   * return <code>docID</code>.
-   */
-  static abstract class DocMap {
-
-    /** Given a doc ID from the original index, return its ordinal in the
-     *  sorted index. */
-    abstract int oldToNew(int docID);
-
-    /** Given the ordinal of a doc ID, return its doc ID in the original index. */
-    abstract int newToOld(int docID);
-
-    /** Return the number of documents in this map. This must be equal to the
-     *  {@link org.apache.lucene.index.LeafReader#maxDoc() number of documents} of the
-     *  {@link org.apache.lucene.index.LeafReader} which is sorted. */
-    abstract int size();
-  }
-
-  /** Check consistency of a {@link DocMap}, useful for assertions. */
-  static boolean isConsistent(DocMap docMap) {
-    final int maxDoc = docMap.size();
-    for (int i = 0; i < maxDoc; ++i) {
-      final int newID = docMap.oldToNew(i);
-      final int oldID = docMap.newToOld(newID);
-      assert newID >= 0 && newID < maxDoc : "doc IDs must be in [0-" + maxDoc + "[, got " + newID;
-      assert i == oldID : "mapping is inconsistent: " + i + " --oldToNew--> " + newID + " --newToOld--> " + oldID;
-      if (i != oldID || newID < 0 || newID >= maxDoc) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /** A comparator of doc IDs. */
-  static abstract class DocComparator {
-
-    /** Compare docID1 against docID2. The contract for the return value is the
-     *  same as {@link Comparator#compare(Object, Object)}. */
-    public abstract int compare(int docID1, int docID2);
-
-  }
-
-  private static final class DocValueSorter extends TimSorter {
-    
-    private final int[] docs;
-    private final Sorter.DocComparator comparator;
-    private final int[] tmp;
-    
-    DocValueSorter(int[] docs, Sorter.DocComparator comparator) {
-      super(docs.length / 64);
-      this.docs = docs;
-      this.comparator = comparator;
-      tmp = new int[docs.length / 64];
-    }
-    
-    @Override
-    protected int compare(int i, int j) {
-      return comparator.compare(docs[i], docs[j]);
-    }
-    
-    @Override
-    protected void swap(int i, int j) {
-      int tmpDoc = docs[i];
-      docs[i] = docs[j];
-      docs[j] = tmpDoc;
-    }
-
-    @Override
-    protected void copy(int src, int dest) {
-      docs[dest] = docs[src];
-    }
-
-    @Override
-    protected void save(int i, int len) {
-      System.arraycopy(docs, i, tmp, 0, len);
-    }
-
-    @Override
-    protected void restore(int i, int j) {
-      docs[j] = tmp[i];
-    }
-
-    @Override
-    protected int compareSaved(int i, int j) {
-      return comparator.compare(tmp[i], docs[j]);
-    }
-  }
-
-  /** Computes the old-to-new permutation over the given comparator. */
-  private static Sorter.DocMap sort(final int maxDoc, DocComparator comparator) {
-    // check if the index is sorted
-    boolean sorted = true;
-    for (int i = 1; i < maxDoc; ++i) {
-      if (comparator.compare(i-1, i) > 0) {
-        sorted = false;
-        break;
-      }
-    }
-    if (sorted) {
-      return null;
-    }
-
-    // sort doc IDs
-    final int[] docs = new int[maxDoc];
-    for (int i = 0; i < maxDoc; i++) {
-      docs[i] = i;
-    }
-    
-    DocValueSorter sorter = new DocValueSorter(docs, comparator);
-    // It can be common to sort a reader, add docs, sort it again, ... and in
-    // that case timSort can save a lot of time
-    sorter.sort(0, docs.length); // docs is now the newToOld mapping
-
-    // The reason why we use MonotonicAppendingLongBuffer here is that it
-    // wastes very little memory if the index is in random order but can save
-    // a lot of memory if the index is already "almost" sorted
-    final PackedLongValues.Builder newToOldBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
-    for (int i = 0; i < maxDoc; ++i) {
-      newToOldBuilder.add(docs[i]);
-    }
-    final PackedLongValues newToOld = newToOldBuilder.build();
-
-    for (int i = 0; i < maxDoc; ++i) {
-      docs[(int) newToOld.get(i)] = i;
-    } // docs is now the oldToNew mapping
-
-    final PackedLongValues.Builder oldToNewBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
-    for (int i = 0; i < maxDoc; ++i) {
-      oldToNewBuilder.add(docs[i]);
-    }
-    final PackedLongValues oldToNew = oldToNewBuilder.build();
-    
-    return new Sorter.DocMap() {
-
-      @Override
-      public int oldToNew(int docID) {
-        return (int) oldToNew.get(docID);
-      }
-
-      @Override
-      public int newToOld(int docID) {
-        return (int) newToOld.get(docID);
-      }
-
-      @Override
-      public int size() {
-        return maxDoc;
-      }
-    };
-  }
-  
-  /**
-   * Returns a mapping from the old document ID to its new location in the
-   * sorted index. Implementations can use the auxiliary
-   * {@link #sort(int, DocComparator)} to compute the old-to-new permutation
-   * given a list of documents and their corresponding values.
-   * <p>
-   * A return value of <tt>null</tt> is allowed and means that
-   * <code>reader</code> is already sorted.
-   * <p>
-   * <b>NOTE:</b> deleted documents are expected to appear in the mapping as
-   * well, they will however be marked as deleted in the sorted view.
-   */
-  DocMap sort(LeafReader reader) throws IOException {
-    SortField fields[] = sort.getSort();
-    final int reverseMul[] = new int[fields.length];
-    final LeafFieldComparator comparators[] = new LeafFieldComparator[fields.length];
-    
-    for (int i = 0; i < fields.length; i++) {
-      reverseMul[i] = fields[i].getReverse() ? -1 : 1;
-      comparators[i] = fields[i].getComparator(1, i).getLeafComparator(reader.getContext());
-      comparators[i].setScorer(FAKESCORER);
-    }
-    final DocComparator comparator = new DocComparator() {
-      @Override
-      public int compare(int docID1, int docID2) {
-        try {
-          for (int i = 0; i < comparators.length; i++) {
-            // TODO: would be better if copy() didnt cause a term lookup in TermOrdVal & co,
-            // the segments are always the same here...
-            comparators[i].copy(0, docID1);
-            comparators[i].setBottom(0);
-            int comp = reverseMul[i] * comparators[i].compareBottom(docID2);
-            if (comp != 0) {
-              return comp;
-            }
-          }
-          return Integer.compare(docID1, docID2); // docid order tiebreak
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-    return sort(reader.maxDoc(), comparator);
-  }
-
-  /**
-   * Returns the identifier of this {@link Sorter}.
-   * <p>This identifier is similar to {@link Object#hashCode()} and should be
-   * chosen so that two instances of this class that sort documents likewise
-   * will have the same identifier. On the contrary, this identifier should be
-   * different on different {@link Sort sorts}.
-   */
-  public String getID() {
-    return sort.toString();
-  }
-
-  @Override
-  public String toString() {
-    return getID();
-  }
-  
-  static final Scorer FAKESCORER = new Scorer(null) {
-
-    float score;
-    int doc = -1;
-    int freq = 1;
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    public DocIdSetIterator iterator() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return freq;
-    }
-
-    @Override
-    public float score() throws IOException {
-      return score;
-    }
-  };
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java
deleted file mode 100644
index 683c5c2..0000000
--- a/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java
+++ /dev/null
@@ -1,940 +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 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}. You can use this class to sort an index as follows:
- *
- * <pre class="prettyprint">
- * IndexWriter writer; // writer to which the sorted index will be added
- * DirectoryReader reader; // reader on the input index
- * Sort sort; // determines how the documents are sorted
- * LeafReader sortingReader = SortingLeafReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
- * writer.addIndexes(reader);
- * writer.close();
- * reader.close();
- * </pre>
- *
- * @lucene.experimental
- */
-public 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;
-    }
-
-    Bits newToOld(final Bits liveDocs) {
-      if (liveDocs == null) {
-        return null;
-      }
-      return new Bits() {
-
-        @Override
-        public boolean get(int index) {
-          return liveDocs.get(docMap.oldToNew(index));
-        }
-
-        @Override
-        public int length() {
-          return liveDocs.length();
-        }
-
-      };
-    }
-
-    @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() {
-      return in.nextOrd();
-    }
-
-    @Override
-    public void setDocument(int docID) {
-      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 {
-      // TODO: this is untested!
-      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/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java b/lucene/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java
deleted file mode 100644
index cd8f84e..0000000
--- a/lucene/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java
+++ /dev/null
@@ -1,264 +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.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.MergePolicy;
-import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.MergeTrigger;
-import org.apache.lucene.index.MultiReader;
-import org.apache.lucene.index.SegmentCommitInfo;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.index.SlowCompositeReaderWrapper;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.packed.PackedInts;
-import org.apache.lucene.util.packed.PackedLongValues;
-
-/** A {@link MergePolicy} that reorders documents according to a {@link Sort}
- *  before merging them. As a consequence, all segments resulting from a merge
- *  will be sorted while segments resulting from a flush will be in the order
- *  in which documents have been added.
- *  <p><b>NOTE</b>: Never use this policy if you rely on
- *  {@link IndexWriter#addDocuments(Iterable) IndexWriter.addDocuments}
- *  to have sequentially-assigned doc IDs, this policy will scatter doc IDs.
- *  <p><b>NOTE</b>: This policy should only be used with idempotent {@code Sort}s 
- *  so that the order of segments is predictable. For example, using 
- *  {@link Sort#INDEXORDER} in reverse (which is not idempotent) will make 
- *  the order of documents in a segment depend on the number of times the segment 
- *  has been merged.
- *  @lucene.experimental */
-public final class SortingMergePolicy extends MergePolicyWrapper {
-
-  /**
-   * Put in the {@link SegmentInfo#getDiagnostics() diagnostics} to denote that
-   * this segment is sorted.
-   */
-  public static final String SORTER_ID_PROP = "sorter";
-  
-  class SortingOneMerge extends OneMerge {
-
-    List<CodecReader> unsortedReaders;
-    Sorter.DocMap docMap;
-    LeafReader sortedView;
-    final InfoStream infoStream;
-
-    SortingOneMerge(List<SegmentCommitInfo> segments, InfoStream infoStream) {
-      super(segments);
-      this.infoStream = infoStream;
-    }
-
-    @Override
-    public List<CodecReader> getMergeReaders() throws IOException {
-      if (unsortedReaders == null) {
-        unsortedReaders = super.getMergeReaders();
-        if (infoStream.isEnabled("SMP")) {
-          infoStream.message("SMP", "sorting " + unsortedReaders);
-          for (LeafReader leaf : unsortedReaders) {
-            String sortDescription = getSortDescription(leaf);
-            if (sortDescription == null) {
-              sortDescription = "not sorted";
-            }
-            infoStream.message("SMP", "seg=" + leaf + " " + sortDescription);
-          }
-        }
-        // wrap readers, to be optimal for merge;
-        List<LeafReader> wrapped = new ArrayList<>(unsortedReaders.size());
-        for (LeafReader leaf : unsortedReaders) {
-          if (leaf instanceof SegmentReader) {
-            leaf = new MergeReaderWrapper((SegmentReader)leaf);
-          }
-          wrapped.add(leaf);
-        }
-        final LeafReader atomicView;
-        if (wrapped.size() == 1) {
-          atomicView = wrapped.get(0);
-        } else {
-          final CompositeReader multiReader = new MultiReader(wrapped.toArray(new LeafReader[wrapped.size()]));
-          atomicView = new SlowCompositeReaderWrapper(multiReader, true);
-        }
-        docMap = sorter.sort(atomicView);
-        sortedView = SortingLeafReader.wrap(atomicView, docMap);
-      }
-      // a null doc map means that the readers are already sorted
-      if (docMap == null) {
-        if (infoStream.isEnabled("SMP")) {
-          infoStream.message("SMP", "readers already sorted, omitting sort");
-        }
-        return unsortedReaders;
-      } else {
-        if (infoStream.isEnabled("SMP")) {
-          infoStream.message("SMP", "sorting readers by " + sort);
-        }
-        return Collections.singletonList(SlowCodecReaderWrapper.wrap(sortedView));
-      }
-    }
-    
-    @Override
-    public void setMergeInfo(SegmentCommitInfo info) {
-      Map<String,String> diagnostics = info.info.getDiagnostics();
-      diagnostics.put(SORTER_ID_PROP, sorter.getID());
-      super.setMergeInfo(info);
-    }
-
-    private PackedLongValues getDeletes(List<CodecReader> readers) {
-      PackedLongValues.Builder deletes = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
-      int deleteCount = 0;
-      for (LeafReader reader : readers) {
-        final int maxDoc = reader.maxDoc();
-        final Bits liveDocs = reader.getLiveDocs();
-        for (int i = 0; i < maxDoc; ++i) {
-          if (liveDocs != null && !liveDocs.get(i)) {
-            ++deleteCount;
-          } else {
-            deletes.add(deleteCount);
-          }
-        }
-      }
-      return deletes.build();
-    }
-
-    @Override
-    public MergePolicy.DocMap getDocMap(final MergeState mergeState) {
-      if (unsortedReaders == null) {
-        throw new IllegalStateException();
-      }
-      if (docMap == null) {
-        return super.getDocMap(mergeState);
-      }
-      assert mergeState.docMaps.length == 1; // we returned a singleton reader
-      final PackedLongValues deletes = getDeletes(unsortedReaders);
-      return new MergePolicy.DocMap() {
-        @Override
-        public int map(int old) {
-          final int oldWithDeletes = old + (int) deletes.get(old);
-          final int newWithDeletes = docMap.oldToNew(oldWithDeletes);
-          return mergeState.docMaps[0].get(newWithDeletes);
-        }
-      };
-    }
-
-    @Override
-    public String toString() {
-      return "SortingMergePolicy.SortingOneMerge(segments=" + segString() + " sort=" + sort + ")";
-    }
-  }
-
-  class SortingMergeSpecification extends MergeSpecification {
-    final InfoStream infoStream;
-    
-    SortingMergeSpecification(InfoStream infoStream) {
-      this.infoStream = infoStream;
-    }
-
-    @Override
-    public void add(OneMerge merge) {
-      super.add(new SortingOneMerge(merge.segments, infoStream));
-    }
-
-    @Override
-    public String segString(Directory dir) {
-      return "SortingMergeSpec(" + super.segString(dir) + ", sorter=" + sorter + ")";
-    }
-
-  }
-
-  /** Returns {@code true} if the given {@code reader} is sorted by the
-   *  {@code sort} given. Typically the given {@code sort} would be the
-   *  {@link SortingMergePolicy#getSort()} order of a {@link SortingMergePolicy}. */
-  public static boolean isSorted(LeafReader reader, Sort sort) {
-    String description = getSortDescription(reader);
-    if (description != null && description.equals(sort.toString())) {
-      return true;
-    }
-    return false;
-  }
-  
-  private static String getSortDescription(LeafReader reader)  {
-    if (reader instanceof SegmentReader) {
-      final SegmentReader segReader = (SegmentReader) reader;
-      final Map<String, String> diagnostics = segReader.getSegmentInfo().info.getDiagnostics();
-      if (diagnostics != null) {
-        return diagnostics.get(SORTER_ID_PROP);
-      }
-    } else if (reader instanceof FilterLeafReader) {
-      return getSortDescription(FilterLeafReader.unwrap(reader));
-    }
-    return null;
-  }
-
-  private MergeSpecification sortedMergeSpecification(MergeSpecification specification, InfoStream infoStream) {
-    if (specification == null) {
-      return null;
-    }
-    MergeSpecification sortingSpec = new SortingMergeSpecification(infoStream);
-    for (OneMerge merge : specification.merges) {
-      sortingSpec.add(merge);
-    }
-    return sortingSpec;
-  }
-
-  final Sorter sorter;
-  final Sort sort;
-
-  /** Create a new {@code MergePolicy} that sorts documents with the given {@code sort}. */
-  public SortingMergePolicy(MergePolicy in, Sort sort) {
-    super(in);
-    this.sorter = new Sorter(sort);
-    this.sort = sort;
-  }
-
-  /** Return the {@link Sort} order that is used to sort segments when merging. */
-  public Sort getSort() {
-    return sort;
-  }
-
-  @Override
-  public MergeSpecification findMerges(MergeTrigger mergeTrigger,
-      SegmentInfos segmentInfos, IndexWriter writer) throws IOException {
-    return sortedMergeSpecification(in.findMerges(mergeTrigger, segmentInfos, writer), writer.infoStream);
-  }
-
-  @Override
-  public MergeSpecification findForcedMerges(SegmentInfos segmentInfos,
-      int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, IndexWriter writer)
-      throws IOException {
-    return sortedMergeSpecification(in.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge, writer), writer.infoStream);
-  }
-
-  @Override
-  public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, IndexWriter writer)
-      throws IOException {
-    return sortedMergeSpecification(in.findForcedDeletesMerges(segmentInfos, writer), writer.infoStream);
-  }
-
-  @Override
-  public String toString() {
-    return "SortingMergePolicy(" + in + ", sorter=" + sorter + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java b/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
index 03a2cb8..7633ff5 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
@@ -20,13 +20,14 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.index.SortingMergePolicy;
 import org.apache.lucene.util.BitSet;
 
+// nocommit what to do here?
+
 /**
  * Helper class to sort readers that contain blocks of documents.
  * <p>
- * Note that this class is intended to used with {@link SortingMergePolicy},
+ * Note that this class is intended to used with index sorting,
  * and for other purposes has some limitations:
  * <ul>
  *    <li>Cannot yet be used with {@link IndexSearcher#searchAfter(ScoreDoc, Query, int, Sort) IndexSearcher.searchAfter}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java b/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
index 5d82be4..731c335 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/EarlyTerminatingSortingCollector.java
@@ -20,14 +20,14 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.SortingMergePolicy;
-import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.FilterLeafCollector;
 import org.apache.lucene.search.FilterCollector;
+import org.apache.lucene.search.FilterLeafCollector;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TotalHitCountCollector;
@@ -39,8 +39,7 @@ import org.apache.lucene.search.TotalHitCountCollector;
  *
  * <p>
  * <b>NOTE:</b> the {@code Collector} detects segments sorted according to a
- * {@link SortingMergePolicy}'s {@link Sort} and so it's best used in conjunction
- * with a {@link SortingMergePolicy}. Also,it collects up to a specified
+ * an {@link IndexWriterConfig#setIndexSort}. Also, it collects up to a specified
  * {@code numDocsToCollect} from each segment, and therefore is mostly suitable
  * for use in conjunction with collectors such as {@link TopDocsCollector}, and
  * not e.g. {@link TotalHitCountCollector}.
@@ -48,24 +47,14 @@ import org.apache.lucene.search.TotalHitCountCollector;
  * <b>NOTE</b>: If you wrap a {@code TopDocsCollector} that sorts in the same
  * order as the index order, the returned {@link TopDocsCollector#topDocs() TopDocs}
  * will be correct. However the total of {@link TopDocsCollector#getTotalHits()
- * hit count} will be underestimated since not all matching documents will have
+ * hit count} will be vastly underestimated since not all matching documents will have
  * been collected.
- * <p>
- * <b>NOTE</b>: This {@code Collector} uses {@link Sort#toString()} to detect
- * whether a segment was sorted with the same {@code Sort}. This has
- * two implications:
- * <ul>
- * <li>if a custom comparator is not implemented correctly and returns
- * different identifiers for equivalent instances, this collector will not
- * detect sorted segments,</li>
- * <li>if you suddenly change the {@link IndexWriter}'s
- * {@code SortingMergePolicy} to sort according to another criterion and if both
- * the old and the new {@code Sort}s have the same identifier, this
- * {@code Collector} will incorrectly detect sorted segments.</li>
- * </ul>
  *
  * @lucene.experimental
  */
+
+// nocommit move to core too
+
 public class EarlyTerminatingSortingCollector extends FilterCollector {
 
   /** Returns whether collection can be early-terminated if it sorts with the
@@ -85,7 +74,6 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
   protected final Sort sort;
   /** Number of documents to collect in each segment */
   protected final int numDocsToCollect;
-  private final Sort mergePolicySort;
   private final AtomicBoolean terminatedEarly = new AtomicBoolean(false);
 
   /**
@@ -99,27 +87,26 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
    *          the number of documents to collect on each segment. When wrapping
    *          a {@link TopDocsCollector}, this number should be the number of
    *          hits.
-   * @param mergePolicySort
-   *          the sort your {@link SortingMergePolicy} uses
    * @throws IllegalArgumentException if the sort order doesn't allow for early
    *          termination with the given merge policy.
    */
-  public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect, Sort mergePolicySort) {
+  public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
     super(in);
     if (numDocsToCollect <= 0) {
       throw new IllegalArgumentException("numDocsToCollect must always be > 0, got " + numDocsToCollect);
     }
-    if (canEarlyTerminate(sort, mergePolicySort) == false) {
-      throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + mergePolicySort);
-    }
     this.sort = sort;
     this.numDocsToCollect = numDocsToCollect;
-    this.mergePolicySort = mergePolicySort;
   }
 
   @Override
   public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
-    if (SortingMergePolicy.isSorted(context.reader(), mergePolicySort)) {
+    Sort segmentSort = context.reader().getIndexSort();
+    if (segmentSort != null && canEarlyTerminate(sort, segmentSort) == false) {
+      throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + segmentSort);
+    }
+
+    if (segmentSort != null) {
       // segment is sorted, can early-terminate
       return new FilterLeafCollector(super.getLeafCollector(context)) {
         private int numCollected;
@@ -142,5 +129,4 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
   public boolean terminatedEarly() {
     return terminatedEarly.get();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/test/org/apache/lucene/index/IndexSortingTest.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/index/IndexSortingTest.java b/lucene/misc/src/test/org/apache/lucene/index/IndexSortingTest.java
deleted file mode 100644
index 8b384f4..0000000
--- a/lucene/misc/src/test/org/apache/lucene/index/IndexSortingTest.java
+++ /dev/null
@@ -1,89 +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.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.SlowCompositeReaderWrapper;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.TestUtil;
-import org.junit.BeforeClass;
-
-public class IndexSortingTest extends SorterTestBase {
-  
-  private static final Sort[] SORT = new Sort[] {
-    new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG)),
-    new Sort(new SortField(null, SortField.Type.DOC, true))
-  };
-  
-  @BeforeClass
-  public static void beforeClassSorterUtilTest() throws Exception {
-    // NOTE: index was created by by super's @BeforeClass
-
-    // only read the values of the undeleted documents, since after addIndexes,
-    // the deleted ones will be dropped from the index.
-    Bits liveDocs = unsortedReader.getLiveDocs();
-    List<Integer> values = new ArrayList<>();
-    for (int i = 0; i < unsortedReader.maxDoc(); i++) {
-      if (liveDocs == null || liveDocs.get(i)) {
-        values.add(Integer.valueOf(unsortedReader.document(i).get(ID_FIELD)));
-      }
-    }
-    int idx = random().nextInt(SORT.length);
-    Sort sorter = SORT[idx];
-    if (idx == 1) { // reverse doc sort
-      Collections.reverse(values);
-    } else {
-      Collections.sort(values);
-      if (random().nextBoolean()) {
-        sorter = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG, true)); // descending
-        Collections.reverse(values);
-      }
-    }
-    sortedValues = values.toArray(new Integer[values.size()]);
-    if (VERBOSE) {
-      System.out.println("sortedValues: " + sortedValues);
-      System.out.println("Sorter: " + sorter);
-    }
-
-    Directory target = newDirectory();
-    IndexWriter writer = new IndexWriter(target, newIndexWriterConfig(null));
-    LeafReader reader = SortingLeafReader.wrap(unsortedReader, sorter);
-    writer.addIndexes(SlowCodecReaderWrapper.wrap(reader));
-    writer.close();
-    // NOTE: also closes unsortedReader
-    reader.close();
-    dir.close();
-    
-    // CheckIndex the target directory
-    dir = target;
-    TestUtil.checkIndex(dir);
-    
-    // set reader for tests
-    sortedReader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
-    assertFalse("index should not have deletions", sortedReader.hasDeletions());
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java b/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
deleted file mode 100644
index df1c80f..0000000
--- a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
+++ /dev/null
@@ -1,405 +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.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
-
-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.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.SortingLeafReader.SortingDocsEnum;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.search.CollectionStatistics;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.TermStatistics;
-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.LuceneTestCase;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.lucene.util.TestUtil;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-public abstract class SorterTestBase extends LuceneTestCase {
-
-  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;
-    }
-  }
-  
-  protected static final String ID_FIELD = "id";
-  protected static final String DOCS_ENUM_FIELD = "docs";
-  protected static final String DOCS_ENUM_TERM = "$all$";
-  protected static final String DOC_POSITIONS_FIELD = "positions";
-  protected static final String DOC_POSITIONS_TERM = "$all$";
-  protected static final String NUMERIC_DV_FIELD = "numeric";
-  protected static final String SORTED_NUMERIC_DV_FIELD = "sorted_numeric";
-  protected static final String NORMS_FIELD = "norm";
-  protected static final String BINARY_DV_FIELD = "binary";
-  protected static final String SORTED_DV_FIELD = "sorted";
-  protected static final String SORTED_SET_DV_FIELD = "sorted_set";
-  protected static final String TERM_VECTORS_FIELD = "term_vectors";
-  protected static final String DIMENSIONAL_FIELD = "numeric1d";
-
-  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();
-  }
-  
-  protected static Directory dir;
-  protected static LeafReader unsortedReader;
-  protected static LeafReader sortedReader;
-  protected static Integer[] sortedValues;
-
-  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);
-    // TODO: index time sorting doesn't yet support points
-    //doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
-    return doc;
-  }
-
-  /** Creates an unsorted index; subclasses then sort this index and open sortedReader. */
-  private static void createIndex(Directory dir, int numDocs, Random random) throws IOException {
-    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));
-    conf.setMaxBufferedDocs(4); // create some segments
-    conf.setSimilarity(new NormsSimilarity(conf.getSimilarity())); // for testing norms field
-    RandomIndexWriter writer = new RandomIndexWriter(random, dir, conf);
-    writer.setDoRandomForceMerge(false);
-    for (int id : ids) {
-      writer.addDocument(doc(id, positions));
-    }
-    // delete some documents
-    writer.commit();
-    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()));
-      }
-    }
-    writer.close();
-  }
-  
-  @BeforeClass
-  public static void beforeClassSorterTestBase() throws Exception {
-    dir = newDirectory();
-    int numDocs = atLeast(20);
-    createIndex(dir, numDocs, random());
-    
-    unsortedReader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
-  }
-  
-  @AfterClass
-  public static void afterClassSorterTestBase() throws Exception {
-    unsortedReader.close();
-    sortedReader.close();
-    dir.close();
-    unsortedReader = sortedReader = null;
-    dir = null;
-  }
-  
-  public void testBinaryDocValuesField() throws Exception {
-    BinaryDocValues dv = sortedReader.getBinaryDocValues(BINARY_DV_FIELD);
-    for (int i = 0; i < sortedReader.maxDoc(); i++) {
-      final BytesRef bytes = dv.get(i);
-      assertEquals("incorrect binary DocValues for doc " + i, sortedValues[i].toString(), bytes.utf8ToString());
-    }
-  }
-  
-  public void testDocsAndPositionsEnum() throws Exception {
-    TermsEnum termsEnum = sortedReader.terms(DOC_POSITIONS_FIELD).iterator();
-    assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
-    PostingsEnum sortedPositions = termsEnum.postings(null, PostingsEnum.ALL);
-    int doc;
-    
-    // test nextDoc()
-    while ((doc = sortedPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      int freq = sortedPositions.freq();
-      assertEquals("incorrect freq for doc=" + doc, sortedValues[doc].intValue() / 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);
-    if (sortedPositions instanceof SortingDocsEnum) {
-      assertTrue(((SortingDocsEnum) sortedPositions).reused(reuse)); // make sure reuse worked
-    }
-    doc = 0;
-    while ((doc = sortedPositions.advance(doc + TestUtil.nextInt(random(), 1, 5))) != DocIdSetIterator.NO_MORE_DOCS) {
-      int freq = sortedPositions.freq();
-      assertEquals("incorrect freq for doc=" + doc, sortedValues[doc].intValue() / 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()));
-      }
-    }
-  }
-
-  Bits randomLiveDocs(int maxDoc) {
-    if (rarely()) {
-      if (random().nextBoolean()) {
-        return null;
-      } else {
-        return new Bits.MatchNoBits(maxDoc);
-      }
-    }
-    final FixedBitSet bits = new FixedBitSet(maxDoc);
-    final int bitsSet = TestUtil.nextInt(random(), 1, maxDoc - 1);
-    for (int i = 0; i < bitsSet; ++i) {
-      while (true) {
-        final int index = random().nextInt(maxDoc);
-        if (!bits.get(index)) {
-          bits.set(index);
-          break;
-        }
-      }
-    }
-    return bits;
-  }
-
-  public void testDocsEnum() throws Exception {
-    TermsEnum termsEnum = sortedReader.terms(DOCS_ENUM_FIELD).iterator();
-    assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOCS_ENUM_TERM)));
-    PostingsEnum docs = termsEnum.postings(null);
-
-    int doc;
-    while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      assertEquals("incorrect value; doc " + doc, sortedValues[doc].intValue(), Integer.parseInt(sortedReader.document(doc).get(ID_FIELD)));
-    }
-
-    PostingsEnum reuse = docs;
-    docs = termsEnum.postings(reuse);
-    if (docs instanceof SortingDocsEnum) {
-      assertTrue(((SortingDocsEnum) docs).reused(reuse)); // make sure reuse worked
-    }
-    doc = -1;
-    while ((doc = docs.advance(doc + 1)) != DocIdSetIterator.NO_MORE_DOCS) {
-      assertEquals("incorrect value; doc " + doc, sortedValues[doc].intValue(), Integer.parseInt(sortedReader.document(doc).get(ID_FIELD)));
-    }
-  }
-  
-  public void testNormValues() throws Exception {
-    NumericDocValues dv = sortedReader.getNormValues(NORMS_FIELD);
-    int maxDoc = sortedReader.maxDoc();
-    for (int i = 0; i < maxDoc; i++) {
-      assertEquals("incorrect norm value for doc " + i, sortedValues[i].intValue(), dv.get(i));
-    }
-  }
-  
-  public void testNumericDocValuesField() throws Exception {
-    NumericDocValues dv = sortedReader.getNumericDocValues(NUMERIC_DV_FIELD);
-    int maxDoc = sortedReader.maxDoc();
-    for (int i = 0; i < maxDoc; i++) {
-      assertEquals("incorrect numeric DocValues for doc " + i, sortedValues[i].intValue(), dv.get(i));
-    }
-  }
-  
-  public void testSortedDocValuesField() throws Exception {
-    SortedDocValues dv = sortedReader.getSortedDocValues(SORTED_DV_FIELD);
-    int maxDoc = sortedReader.maxDoc();
-    for (int i = 0; i < maxDoc; i++) {
-      final BytesRef bytes = dv.get(i);
-      assertEquals("incorrect sorted DocValues for doc " + i, sortedValues[i].toString(), bytes.utf8ToString());
-    }
-  }
-  
-  public void testSortedSetDocValuesField() throws Exception {
-    SortedSetDocValues dv = sortedReader.getSortedSetDocValues(SORTED_SET_DV_FIELD);
-    int maxDoc = sortedReader.maxDoc();
-    for (int i = 0; i < maxDoc; i++) {
-      dv.setDocument(i);
-      BytesRef bytes = dv.lookupOrd(dv.nextOrd());
-      int value = sortedValues[i].intValue();
-      assertEquals("incorrect sorted-set DocValues for doc " + i, Integer.valueOf(value).toString(), bytes.utf8ToString());
-      bytes = dv.lookupOrd(dv.nextOrd());
-      assertEquals("incorrect sorted-set DocValues for doc " + i, Integer.valueOf(value + 1).toString(), bytes.utf8ToString());
-      assertEquals(SortedSetDocValues.NO_MORE_ORDS, dv.nextOrd());
-    }
-  }
-  
-  public void testSortedNumericDocValuesField() throws Exception {
-    SortedNumericDocValues dv = sortedReader.getSortedNumericDocValues(SORTED_NUMERIC_DV_FIELD);
-    int maxDoc = sortedReader.maxDoc();
-    for (int i = 0; i < maxDoc; i++) {
-      dv.setDocument(i);
-      assertEquals(2, dv.count());
-      int value = sortedValues[i].intValue();
-      assertEquals("incorrect sorted-numeric DocValues for doc " + i, value, dv.valueAt(0));
-      assertEquals("incorrect sorted-numeric DocValues for doc " + i, value + 1, dv.valueAt(1));
-    }
-  }
-  
-  public void testTermVectors() throws Exception {
-    int maxDoc = sortedReader.maxDoc();
-    for (int i = 0; i < maxDoc; i++) {
-      Terms terms = sortedReader.getTermVector(i, TERM_VECTORS_FIELD);
-      assertNotNull("term vectors not found for doc " + i + " field [" + TERM_VECTORS_FIELD + "]", terms);
-      assertEquals("incorrect term vector for doc " + i, sortedValues[i].toString(), terms.iterator().next().utf8ToString());
-    }
-  }
-
-  // TODO: index sorting doesn't yet support points
-  /*
-  public void testPoints() throws Exception {
-    PointValues values = sortedReader.getPointValues();
-    values.intersect(DIMENSIONAL_FIELD,
-                     new IntersectVisitor() {
-                       @Override
-                       public void visit(int docID) {
-                         throw new IllegalStateException();
-                       }
-
-                       @Override
-                       public void visit(int docID, byte[] packedValues) {
-                         assertEquals(sortedValues[docID].intValue(), NumericUtils.bytesToInt(packedValues, 0));
-                       }
-
-                       @Override
-                       public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-                         return Relation.CELL_CROSSES_QUERY;
-                       }
-                     });
-  }
-  */
-}