You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2014/12/05 15:43:53 UTC

svn commit: r1643299 - in /lucene/dev/branches/lucene2878/lucene: codecs/src/java/org/apache/lucene/codecs/memory/ core/src/java/org/apache/lucene/search/ core/src/java/org/apache/lucene/search/posfilter/ join/src/java/org/apache/lucene/search/join/ mi...

Author: romseygeek
Date: Fri Dec  5 14:43:53 2014
New Revision: 1643299

URL: http://svn.apache.org/r1643299
Log:
Fix final test failures

Modified:
    lucene/dev/branches/lucene2878/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
    lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/NonOverlappingQuery.java
    lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
    lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java

Modified: lucene/dev/branches/lucene2878/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1643299&r1=1643298&r2=1643299&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2878/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Fri Dec  5 14:43:53 2014
@@ -332,8 +332,9 @@ public final class MemoryPostingsFormat
         } else if (writeOffsets == false) {
           if (writePayloads) {
             enumFlags = DocsEnum.FLAG_PAYLOADS;
-          } else {
-            enumFlags = 0;
+          }
+          else {
+            enumFlags = DocsEnum.FLAG_POSITIONS;
           }
         } else {
           if (writePayloads) {

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1643299&r1=1643298&r2=1643299&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri Dec  5 14:43:53 2014
@@ -27,132 +27,99 @@ import org.apache.lucene.util.BytesRef;
 
 /** Scorer for conjunctions, sets of queries, all of which are required. */
 class ConjunctionScorer extends Scorer {
-  
-  private final Scorer[] scorersOrdered;
-  private final Scorer[] scorers;
-  private int lastDoc = -1;
+
+  protected int lastDoc = -1;
+  protected final DocsAndFreqs[] docsAndFreqs;
+  private final DocsAndFreqs lead;
   private final float coord;
-  final PositionQueue posQueue;
+  private final PositionQueue posQueue;
+  private final Scorer[] scorers; // to preserve order for positional queries
 
-  public ConjunctionScorer(Weight weight, Scorer[] scorers) throws IOException {
+  ConjunctionScorer(Weight weight, Scorer[] scorers) {
     this(weight, scorers, 1f);
   }
-  
-  public ConjunctionScorer(Weight weight, Scorer[] scorers, float coord) throws IOException {
+
+  ConjunctionScorer(Weight weight, Scorer[] scorers, float coord) {
     super(weight);
-    scorersOrdered = new Scorer[scorers.length];
-    System.arraycopy(scorers, 0, scorersOrdered, 0, scorers.length);
-    this.scorers = scorers;
     this.coord = coord;
-    posQueue = new PositionQueue(scorers);
-    
+    this.docsAndFreqs = new DocsAndFreqs[scorers.length];
     for (int i = 0; i < scorers.length; i++) {
-      if (scorers[i].nextDoc() == NO_MORE_DOCS) {
-        // If even one of the sub-scorers does not have any documents, this
-        // scorer should not attempt to do any more work.
-        lastDoc = NO_MORE_DOCS;
-        return;
-      }
+      docsAndFreqs[i] = new DocsAndFreqs(scorers[i]);
     }
-
-    // Sort the array the first time...
-    // We don't need to sort the array in any future calls because we know
-    // it will already start off sorted (all scorers on same doc).
-    
-    // Note that this comparator is not consistent with equals!
-    // Also we use mergeSort here to be stable (so order of Scorers that
-    // match on first document keeps preserved):
-    ArrayUtil.timSort(scorers, new Comparator<Scorer>() { // sort the array
+    // Sort the array the first time to allow the least frequent DocsEnum to
+    // lead the matching.
+    ArrayUtil.timSort(docsAndFreqs, new Comparator<DocsAndFreqs>() {
       @Override
-      public int compare(Scorer o1, Scorer o2) {
-        return o1.docID() - o2.docID();
+      public int compare(DocsAndFreqs o1, DocsAndFreqs o2) {
+        return Long.compare(o1.cost, o2.cost);
       }
     });
 
-    // NOTE: doNext() must be called before the re-sorting of the array later on.
-    // The reason is this: assume there are 5 scorers, whose first docs are 1,
-    // 2, 3, 5, 5 respectively. Sorting (above) leaves the array as is. Calling
-    // doNext() here advances all the first scorers to 5 (or a larger doc ID
-    // they all agree on). 
-    // However, if we re-sort before doNext() is called, the order will be 5, 3,
-    // 2, 1, 5 and then doNext() will stop immediately, since the first scorer's
-    // docs equals the last one. So the invariant that after calling doNext() 
-    // all scorers are on the same doc ID is broken.
-    if (doNext() == NO_MORE_DOCS) {
-      // The scorers did not agree on any document.
-      lastDoc = NO_MORE_DOCS;
-      return;
-    }
-
-    // If first-time skip distance is any predictor of
-    // scorer sparseness, then we should always try to skip first on
-    // those scorers.
-    // Keep last scorer in it's last place (it will be the first
-    // to be skipped on), but reverse all of the others so that
-    // they will be skipped on in order of original high skip.
-    int end = scorers.length - 1;
-    int max = end >> 1;
-    for (int i = 0; i < max; i++) {
-      Scorer tmp = scorers[i];
-      int idx = end - i - 1;
-      scorers[i] = scorers[idx];
-      scorers[idx] = tmp;
-    }
+    lead = docsAndFreqs[0]; // least frequent DocsEnum leads the intersection
+    posQueue = new PositionQueue(scorers);
+    this.scorers = scorers;
   }
 
-  private int doNext() throws IOException {
-    int first = 0;
-    int doc = scorers[scorers.length - 1].docID();
-    Scorer firstScorer;
-    while ((firstScorer = scorers[first]).docID() < doc) {
-      doc = firstScorer.advance(doc);
-      first = first == scorers.length - 1 ? 0 : first + 1;
+  private int doNext(int doc) throws IOException {
+    for(;;) {
+      // doc may already be NO_MORE_DOCS here, but we don't check explicitly
+      // since all scorers should advance to NO_MORE_DOCS, match, then
+      // return that value.
+      advanceHead: for(;;) {
+        for (int i = 1; i < docsAndFreqs.length; i++) {
+          // invariant: docsAndFreqs[i].doc <= doc at this point.
+
+          // docsAndFreqs[i].doc may already be equal to doc if we "broke advanceHead"
+          // on the previous iteration and the advance on the lead scorer exactly matched.
+          if (docsAndFreqs[i].doc < doc) {
+            docsAndFreqs[i].doc = docsAndFreqs[i].scorer.advance(doc);
+
+            if (docsAndFreqs[i].doc > doc) {
+              // DocsEnum beyond the current doc - break and advance lead to the new highest doc.
+              doc = docsAndFreqs[i].doc;
+              break advanceHead;
+            }
+          }
+        }
+        // success - all DocsEnums are on the same doc
+        posQueue.advanceTo(doc);
+        return doc;
+      }
+      // advance head for next iteration
+      doc = lead.doc = lead.scorer.advance(doc);
     }
-    posQueue.advanceTo(doc);
-    return doc;
   }
-  
+
   @Override
   public int advance(int target) throws IOException {
-    if (lastDoc == NO_MORE_DOCS) {
-      return lastDoc;
-    } else if (scorers[(scorers.length - 1)].docID() < target) {
-      scorers[(scorers.length - 1)].advance(target);
-    }
-    return lastDoc = doNext();
+    lead.doc = lead.scorer.advance(target);
+    return lastDoc = doNext(lead.doc);
   }
 
   @Override
   public int docID() {
     return lastDoc;
   }
-  
+
   @Override
   public int nextDoc() throws IOException {
-    if (lastDoc == NO_MORE_DOCS) {
-      return lastDoc;
-    } else if (lastDoc == -1) {
-      lastDoc = scorers[scorers.length - 1].docID();
-      posQueue.advanceTo(lastDoc);
-      return lastDoc;
-    }
-    scorers[(scorers.length - 1)].nextDoc();
-    return lastDoc = doNext();
+    lead.doc = lead.scorer.nextDoc();
+    return lastDoc = doNext(lead.doc);
   }
-  
+
   @Override
   public float score() throws IOException {
     // TODO: sum into a double and cast to float if we ever send required clauses to BS1
     float sum = 0.0f;
-    for (int i = 0; i < scorers.length; i++) {
-      sum += scorers[i].score();
+    for (DocsAndFreqs docs : docsAndFreqs) {
+      sum += docs.scorer.score();
     }
     return sum * coord;
   }
 
   @Override
-  public int freq() throws IOException {
-    return scorers.length;
+  public int freq() {
+    return docsAndFreqs.length;
   }
 
   @Override
@@ -187,19 +154,26 @@ class ConjunctionScorer extends Scorer {
 
   @Override
   public long cost() {
-    long sum = 0;
-    for (int i = 0; i < scorers.length; i++) {
-      sum += scorers[i].cost();
-    }
-    return sum; // nocommit is this right?
+    return lead.scorer.cost();
   }
 
   @Override
   public Collection<ChildScorer> getChildren() {
-    ArrayList<ChildScorer> children = new ArrayList<ChildScorer>(scorers.length);
-    for (Scorer scorer : scorersOrdered) {
+    ArrayList<ChildScorer> children = new ArrayList<>(scorers.length);
+    for (Scorer scorer : scorers) {
       children.add(new ChildScorer(scorer, "MUST"));
     }
     return children;
   }
+
+  static final class DocsAndFreqs {
+    final long cost;
+    final Scorer scorer;
+    int doc = -1;
+
+    DocsAndFreqs(Scorer scorer) {
+      this.scorer = scorer;
+      this.cost = scorer.cost();
+    }
+  }
 }

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1643299&r1=1643298&r2=1643299&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Fri Dec  5 14:43:53 2014
@@ -92,7 +92,6 @@ final class SloppyPhraseScorer extends P
   private int matchLength;
   private int startpos = -1;
   private int startoffset = -1;
-  private int startoffset = -1;
   private int endoffset = -1;
 
   @Override
@@ -153,8 +152,6 @@ final class SloppyPhraseScorer extends P
   private void setSpanStart(int topPos, int topStartOffset) throws IOException {
     startpos = topPos;
     startoffset = topStartOffset;
-    startoffset = topStartOffset;
-    endoffset = topEndOffset;
     for (Object o : pq.getPPs()) {
       if (o == null)
         continue;

Modified: lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/NonOverlappingQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/NonOverlappingQuery.java?rev=1643299&r1=1643298&r2=1643299&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/NonOverlappingQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/core/src/java/org/apache/lucene/search/posfilter/NonOverlappingQuery.java Fri Dec  5 14:43:53 2014
@@ -17,6 +17,9 @@ package org.apache.lucene.search.posfilt
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Set;
+
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
@@ -27,9 +30,6 @@ import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Bits;
 
-import java.io.IOException;
-import java.util.Set;
-
 /**
  * A Query that matches documents containing an interval (the minuend) that
  * does not contain another interval (the subtrahend).
@@ -148,7 +148,7 @@ public final class NonOverlappingQuery e
     @Override
     protected void reset(int doc) throws IOException {
       super.reset(doc);
-      if (this.subtrahend == null || this.subtrahend.advance(doc) != doc)
+      if (this.subtrahend == null || this.subtrahend.docID() == NO_MORE_DOCS || this.subtrahend.advance(doc) != doc)
         subtPosition = NO_MORE_POSITIONS;
       else
         subtPosition = -1;

Modified: lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java?rev=1643299&r1=1643298&r2=1643299&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java (original)
+++ lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java Fri Dec  5 14:43:53 2014
@@ -17,15 +17,21 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Queue;
+
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.FakeScorer;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.FieldValueHitQueue;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreCachingWrappingScorer;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.IndexWriter; // javadocs
-import org.apache.lucene.search.*;
 import org.apache.lucene.search.Scorer.ChildScorer;
 import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.Sort;
@@ -37,13 +43,6 @@ import org.apache.lucene.search.grouping
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.util.ArrayUtil;
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Queue;
-
 
 /** Collects parent document hits for a Query containing one more more
  *  BlockJoinQuery clauses, sorted by the

Modified: lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java?rev=1643299&r1=1643298&r2=1643299&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java (original)
+++ lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java Fri Dec  5 14:43:53 2014
@@ -48,13 +48,10 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 
-import java.io.IOException;
-import java.util.Arrays;
-
 /**
  * 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
@@ -64,7 +61,7 @@ import java.util.Arrays;
  * writer.close();
  * reader.close();
  * </pre>
- * 
+ *
  * @lucene.experimental
  */
 public class SortingLeafReader extends FilterLeafReader {
@@ -96,7 +93,7 @@ public class SortingLeafReader extends F
 
     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;
@@ -120,7 +117,7 @@ public class SortingLeafReader extends F
 
     final Sorter.DocMap docMap; // pkg-protected to avoid synthetic accessor methods
     private final IndexOptions indexOptions;
-    
+
     public SortingTermsEnum(final TermsEnum in, Sorter.DocMap docMap, IndexOptions indexOptions) {
       super(in);
       this.docMap = docMap;
@@ -148,6 +145,10 @@ public class SortingLeafReader extends F
 
     @Override
     public DocsEnum docs(Bits liveDocs, DocsEnum reuse, final int flags) throws IOException {
+
+      if ((flags & DocsEnum.FLAG_POSITIONS) >= DocsEnum.FLAG_POSITIONS)
+        return docsAndPositions(liveDocs, reuse, flags);
+
       final DocsEnum inReuse;
       final SortingDocsEnum wrapReuse;
       if (reuse != null && reuse instanceof SortingDocsEnum) {
@@ -167,16 +168,38 @@ public class SortingLeafReader extends F
 
     @Override
     public DocsEnum docsAndPositions(Bits liveDocs, DocsEnum reuse, final int flags) throws IOException {
-      return docs(liveDocs, reuse, flags);
+      final DocsEnum inReuse;
+      final SortingDocsAndPositionsEnum wrapReuse;
+      if (reuse != null && reuse instanceof SortingDocsAndPositionsEnum) {
+        // if we're asked to reuse the given DocsEnum and it is Sorting, return
+        // the wrapped one, since some Codecs expect it.
+        wrapReuse = (SortingDocsAndPositionsEnum) reuse;
+        inReuse = wrapReuse.getWrapped();
+      } else {
+        wrapReuse = null;
+        inReuse = reuse;
+      }
+
+      final DocsEnum inDocsAndPositions = in.docsAndPositions(newToOld(liveDocs), inReuse, flags);
+      if (inDocsAndPositions == null) {
+        return null;
+      }
+
+      // 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 SortingDocsAndPositionsEnum(docMap.size(), wrapReuse, inDocsAndPositions, docMap, storeOffsets);
     }
 
   }
 
   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;
@@ -187,7 +210,7 @@ public class SortingLeafReader extends F
       return in.get(docMap.newToOld(docID));
     }
   }
-  
+
   private static class SortingNumericDocValues extends NumericDocValues {
 
     private final NumericDocValues in;
@@ -203,33 +226,33 @@ public class SortingLeafReader extends F
       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;
@@ -250,12 +273,12 @@ public class SortingLeafReader extends F
       return in.length();
     }
   }
-  
+
   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;
@@ -286,12 +309,12 @@ public class SortingLeafReader extends F
       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;
@@ -324,14 +347,14 @@ public class SortingLeafReader extends F
   }
 
   static class SortingDocsEnum extends FilterDocsEnum {
-    
+
     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];
@@ -349,13 +372,13 @@ public class SortingLeafReader extends F
       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];
@@ -463,44 +486,43 @@ public class SortingLeafReader extends F
       // 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 DocsEnum}. */
     DocsEnum getWrapped() {
       return in;
     }
   }
 
-  // nocommit - merge this into SortingDocsEnum?
   static class SortingDocsAndPositionsEnum extends FilterDocsEnum {
-    
+
     /**
      * A {@link TimSorter} which sorts two parallel arrays of doc IDs and
      * offsets in one go. Everytime a doc ID is 'swapped', its correponding 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];
@@ -516,13 +538,13 @@ public class SortingLeafReader extends F
       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;
@@ -551,16 +573,16 @@ public class SortingLeafReader extends F
         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;
@@ -629,7 +651,7 @@ public class SortingLeafReader extends F
         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. 
+        // previous bits are the delta-encoded position.
         final int token = (pos - previousPosition) << 1 | (payload == null ? 0 : 1);
         out.writeVInt(token);
         previousPosition = pos;
@@ -646,34 +668,34 @@ public class SortingLeafReader extends F
         }
       }
     }
-    
+
     @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;
@@ -684,7 +706,7 @@ public class SortingLeafReader extends F
       endOffset = 0;
       return docs[docIt];
     }
-    
+
     @Override
     public int nextPosition() throws IOException {
       final int token = postingInput.readVInt();
@@ -705,13 +727,13 @@ public class SortingLeafReader extends F
       }
       return pos;
     }
-    
+
     @Override
     public int startOffset() throws IOException {
       return startOffset;
     }
 
-    /** Returns the wrapped {@link org.apache.lucene.index.DocsEnum}. */
+    /** Returns the wrapped {@link DocsEnum}. */
     DocsEnum getWrapped() {
       return in;
     }
@@ -748,12 +770,12 @@ public class SortingLeafReader extends F
   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);
@@ -763,7 +785,7 @@ public class SortingLeafReader extends F
       return new SortingBinaryDocValues(oldDocValues, docMap);
     }
   }
-  
+
   @Override
   public Bits getLiveDocs() {
     final Bits inLiveDocs = in.getLiveDocs();
@@ -773,7 +795,7 @@ public class SortingLeafReader extends F
       return new SortingBits(inLiveDocs, docMap);
     }
   }
-  
+
   @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     final NumericDocValues norm = in.getNormValues(field);
@@ -790,7 +812,7 @@ public class SortingLeafReader extends F
     if (oldDocValues == null) return null;
     return new SortingNumericDocValues(oldDocValues, docMap);
   }
-  
+
   @Override
   public SortedNumericDocValues getSortedNumericDocValues(String field)
       throws IOException {
@@ -811,7 +833,7 @@ public class SortingLeafReader extends F
       return new SortingSortedDocValues(sortedDV, docMap);
     }
   }
-  
+
   @Override
   public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
     SortedSetDocValues sortedSetDV = in.getSortedSetDocValues(field);
@@ -819,7 +841,7 @@ public class SortingLeafReader extends F
       return null;
     } else {
       return new SortingSortedSetDocValues(sortedSetDV, docMap);
-    }  
+    }
   }
 
   @Override
@@ -836,5 +858,5 @@ public class SortingLeafReader extends F
   public Fields getTermVectors(final int docID) throws IOException {
     return in.getTermVectors(docMap.newToOld(docID));
   }
-  
+
 }