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 2015/02/09 00:53:25 UTC

svn commit: r1658277 [11/38] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/lucene/replicator/ dev-tools/maven/solr/ dev-tools/maven/solr/contrib/...

Modified: lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java Sun Feb  8 23:53:14 2015
@@ -133,21 +133,25 @@ class TermsIncludingScoreQuery extends Q
 
       @Override
       public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-        SVInnerScorer scorer = (SVInnerScorer) bulkScorer(context, false, null);
-        if (scorer != null) {
-          return scorer.explain(doc);
+        Terms terms = context.reader().terms(field);
+        if (terms != null) {
+          segmentTermsEnum = terms.iterator(segmentTermsEnum);
+          BytesRef spare = new BytesRef();
+          DocsEnum docsEnum = null;
+          for (int i = 0; i < TermsIncludingScoreQuery.this.terms.size(); i++) {
+            if (segmentTermsEnum.seekExact(TermsIncludingScoreQuery.this.terms.get(ords[i], spare))) {
+              docsEnum = segmentTermsEnum.docs(null, docsEnum, DocsEnum.FLAG_NONE);
+              if (docsEnum.advance(doc) == doc) {
+                final float score = TermsIncludingScoreQuery.this.scores[ords[i]];
+                return new ComplexExplanation(true, score, "Score based on join value " + segmentTermsEnum.term().utf8ToString());
+              }
+            }
+          }
         }
         return new ComplexExplanation(false, 0.0f, "Not a match");
       }
 
       @Override
-      public boolean scoresDocsOutOfOrder() {
-        // We have optimized impls below if we are allowed
-        // to score out-of-order:
-        return true;
-      }
-
-      @Override
       public Query getQuery() {
         return TermsIncludingScoreQuery.this;
       }
@@ -179,143 +183,9 @@ class TermsIncludingScoreQuery extends Q
           return new SVInOrderScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
         }
       }
-
-      @Override
-      public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
-
-        if (scoreDocsInOrder) {
-          return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
-        } else {
-          Terms terms = context.reader().terms(field);
-          if (terms == null) {
-            return null;
-          }
-          // what is the runtime...seems ok?
-          final long cost = context.reader().maxDoc() * terms.size();
-
-          segmentTermsEnum = terms.iterator(segmentTermsEnum);
-          // Optimized impls that take advantage of docs
-          // being allowed to be out of order:
-          if (multipleValuesPerDocument) {
-            return new MVInnerScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
-          } else {
-            return new SVInnerScorer(this, acceptDocs, segmentTermsEnum, cost);
-          }
-        }
-      }
     };
   }
 
-  // This impl assumes that the 'join' values are used uniquely per doc per field. Used for one to many relations.
-  class SVInnerScorer extends BulkScorer {
-
-    final BytesRef spare = new BytesRef();
-    final Bits acceptDocs;
-    final TermsEnum termsEnum;
-    final long cost;
-
-    int upto;
-    DocsEnum docsEnum;
-    DocsEnum reuse;
-    int scoreUpto;
-    int doc;
-
-    SVInnerScorer(Weight weight, Bits acceptDocs, TermsEnum termsEnum, long cost) {
-      this.acceptDocs = acceptDocs;
-      this.termsEnum = termsEnum;
-      this.cost = cost;
-      this.doc = -1;
-    }
-
-    @Override
-    public boolean score(LeafCollector collector, int max) throws IOException {
-      FakeScorer fakeScorer = new FakeScorer();
-      collector.setScorer(fakeScorer);
-      if (doc == -1) {
-        doc = nextDocOutOfOrder();
-      }
-      while(doc < max) {
-        fakeScorer.doc = doc;
-        fakeScorer.score = scores[ords[scoreUpto]];
-        collector.collect(doc);
-        doc = nextDocOutOfOrder();
-      }
-
-      return doc != DocsEnum.NO_MORE_DOCS;
-    }
-
-    int nextDocOutOfOrder() throws IOException {
-      while (true) {
-        if (docsEnum != null) {
-          int docId = docsEnumNextDoc();
-          if (docId == DocIdSetIterator.NO_MORE_DOCS) {
-            docsEnum = null;
-          } else {
-            return doc = docId;
-          }
-        }
-
-        if (upto == terms.size()) {
-          return doc = DocIdSetIterator.NO_MORE_DOCS;
-        }
-
-        scoreUpto = upto;
-        if (termsEnum.seekExact(terms.get(ords[upto++], spare))) {
-          docsEnum = reuse = termsEnum.docs(acceptDocs, reuse, DocsEnum.FLAG_NONE);
-        }
-      }
-    }
-
-    protected int docsEnumNextDoc() throws IOException {
-      return docsEnum.nextDoc();
-    }
-
-    private Explanation explain(int target) throws IOException {
-      int docId;
-      do {
-        docId = nextDocOutOfOrder();
-        if (docId < target) {
-          int tempDocId = docsEnum.advance(target);
-          if (tempDocId == target) {
-            docId = tempDocId;
-            break;
-          }
-        } else if (docId == target) {
-          break;
-        }
-        docsEnum = null; // goto the next ord.
-      } while (docId != DocIdSetIterator.NO_MORE_DOCS);
-
-      return new ComplexExplanation(true, scores[ords[scoreUpto]], "Score based on join value " + termsEnum.term().utf8ToString());
-    }
-  }
-
-  // This impl that tracks whether a docid has already been emitted. This check makes sure that docs aren't emitted
-  // twice for different join values. This means that the first encountered join value determines the score of a document
-  // even if other join values yield a higher score.
-  class MVInnerScorer extends SVInnerScorer {
-
-    final FixedBitSet alreadyEmittedDocs;
-
-    MVInnerScorer(Weight weight, Bits acceptDocs, TermsEnum termsEnum, int maxDoc, long cost) {
-      super(weight, acceptDocs, termsEnum, cost);
-      alreadyEmittedDocs = new FixedBitSet(maxDoc);
-    }
-
-    @Override
-    protected int docsEnumNextDoc() throws IOException {
-      while (true) {
-        int docId = docsEnum.nextDoc();
-        if (docId == DocIdSetIterator.NO_MORE_DOCS) {
-          return docId;
-        }
-        if (!alreadyEmittedDocs.getAndSet(docId)) {
-          return docId;//if it wasn't previously set, return it
-        }
-      }
-    }
-  }
-
   class SVInOrderScorer extends Scorer {
 
     final DocIdSetIterator matchingDocsIterator;

Modified: lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java Sun Feb  8 23:53:14 2015
@@ -57,11 +57,6 @@ abstract class TermsWithScoreCollector e
     this.scorer = scorer;
   }
 
-  @Override
-  public boolean acceptsDocsOutOfOrder() {
-    return true;
-  }
-
   /**
    * Chooses the right {@link TermsWithScoreCollector} implementation.
    *

Modified: lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java Sun Feb  8 23:53:14 2015
@@ -154,11 +154,6 @@ public class ToChildBlockJoinQuery exten
       throw new UnsupportedOperationException(getClass().getName() +
                                               " cannot explain match on parent document");
     }
-
-    @Override
-    public boolean scoresDocsOutOfOrder() {
-      return false;
-    }
   }
 
   static class ToChildBlockJoinScorer extends Scorer {

Modified: lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java Sun Feb  8 23:53:14 2015
@@ -38,6 +38,9 @@ import java.util.*;
  *  IndexWriter#updateDocuments}).  Ie, the join is computed
  *  at index time.
  *
+ *  <p>This collector MUST be used with {@link ToParentBlockJoinIndexSearcher},
+ *  in order to work correctly.
+ *
  *  <p>The parent Sort must only use
  *  fields from the parent documents; sorting by field in
  *  the child documents is not supported.</p>
@@ -74,7 +77,7 @@ import java.util.*;
  *
  * @lucene.experimental
  */
-public class ToParentBlockJoinCollector extends SimpleCollector {
+public class ToParentBlockJoinCollector implements Collector {
 
   private final Sort sort;
 
@@ -83,16 +86,11 @@ public class ToParentBlockJoinCollector
   private final Map<Query,Integer> joinQueryID = new HashMap<>();
   private final int numParentHits;
   private final FieldValueHitQueue<OneGroup> queue;
-  private final FieldComparator[] comparators;
-  private final int[] reverseMul;
-  private final int compEnd;
+  private final FieldComparator<?>[] comparators;
   private final boolean trackMaxScore;
   private final boolean trackScores;
 
-  private int docBase;
   private ToParentBlockJoinQuery.BlockJoinScorer[] joinScorers = new ToParentBlockJoinQuery.BlockJoinScorer[0];
-  private LeafReaderContext currentReaderContext;
-  private Scorer scorer;
   private boolean queueFull;
 
   private OneGroup bottom;
@@ -116,8 +114,6 @@ public class ToParentBlockJoinCollector
     this.numParentHits = numParentHits;
     queue = FieldValueHitQueue.create(sort.getSort(), numParentHits);
     comparators = queue.getComparators();
-    reverseMul = queue.getReverseMul();
-    compEnd = comparators.length - 1;
   }
   
   private static final class OneGroup extends FieldValueHitQueue.Entry {
@@ -143,143 +139,167 @@ public class ToParentBlockJoinCollector
   }
 
   @Override
-  public void collect(int parentDoc) throws IOException {
-    //System.out.println("\nC parentDoc=" + parentDoc);
-    totalHitCount++;
-
-    float score = Float.NaN;
+  public LeafCollector getLeafCollector(final LeafReaderContext context)
+      throws IOException {
+    final LeafFieldComparator[] comparators = queue.getComparators(context);
+    final int[] reverseMul = queue.getReverseMul();
+    final int docBase = context.docBase;
+    return new LeafCollector() {
+
+      private Scorer scorer;
+
+      @Override
+      public void setScorer(Scorer scorer) throws IOException {
+        //System.out.println("C.setScorer scorer=" + scorer);
+        // Since we invoke .score(), and the comparators likely
+        // do as well, cache it so it's only "really" computed
+        // once:
+        if (scorer instanceof ScoreCachingWrappingScorer == false) {
+          scorer = new ScoreCachingWrappingScorer(scorer);
+        }
+        this.scorer = scorer;
+        for (LeafFieldComparator comparator : comparators) {
+          comparator.setScorer(scorer);
+        }
+        Arrays.fill(joinScorers, null);
 
-    if (trackMaxScore) {
-      score = scorer.score();
-      maxScore = Math.max(maxScore, score);
-    }
+        Queue<Scorer> queue = new LinkedList<>();
+        //System.out.println("\nqueue: add top scorer=" + scorer);
+        queue.add(scorer);
+        while ((scorer = queue.poll()) != null) {
+          //System.out.println("  poll: " + scorer + "; " + scorer.getWeight().getQuery());
+          if (scorer instanceof ToParentBlockJoinQuery.BlockJoinScorer) {
+            enroll((ToParentBlockJoinQuery) scorer.getWeight().getQuery(), (ToParentBlockJoinQuery.BlockJoinScorer) scorer);
+          }
 
-    // TODO: we could sweep all joinScorers here and
-    // aggregate total child hit count, so we can fill this
-    // in getTopGroups (we wire it to 0 now)
-
-    if (queueFull) {
-      //System.out.println("  queueFull");
-      // Fastmatch: return if this hit is not competitive
-      for (int i = 0;; i++) {
-        final int c = reverseMul[i] * comparators[i].compareBottom(parentDoc);
-        if (c < 0) {
-          // Definitely not competitive.
-          //System.out.println("    skip");
-          return;
-        } else if (c > 0) {
-          // Definitely competitive.
-          break;
-        } else if (i == compEnd) {
-          // Here c=0. If we're at the last comparator, this doc is not
-          // competitive, since docs are visited in doc Id order, which means
-          // this doc cannot compete with any other document in the queue.
-          //System.out.println("    skip");
-          return;
+          for (ChildScorer sub : scorer.getChildren()) {
+            //System.out.println("  add sub: " + sub.child + "; " + sub.child.getWeight().getQuery());
+            queue.add(sub.child);
+          }
         }
       }
+      
+      @Override
+      public void collect(int parentDoc) throws IOException {
+      //System.out.println("\nC parentDoc=" + parentDoc);
+        totalHitCount++;
+
+        float score = Float.NaN;
+
+        if (trackMaxScore) {
+          score = scorer.score();
+          maxScore = Math.max(maxScore, score);
+        }
 
-      //System.out.println("    competes!  doc=" + (docBase + parentDoc));
+        // TODO: we could sweep all joinScorers here and
+        // aggregate total child hit count, so we can fill this
+        // in getTopGroups (we wire it to 0 now)
+
+        if (queueFull) {
+          //System.out.println("  queueFull");
+          // Fastmatch: return if this hit is not competitive
+          int c = 0;
+          for (int i = 0; i < comparators.length; ++i) {
+            c = reverseMul[i] * comparators[i].compareBottom(parentDoc);
+            if (c != 0) {
+              break;
+            }
+          }
+          if (c <= 0) { // in case of equality, this hit is not competitive as docs are visited in order
+            // Definitely not competitive.
+            //System.out.println("    skip");
+            return;
+          }
 
-      // This hit is competitive - replace bottom element in queue & adjustTop
-      for (int i = 0; i < comparators.length; i++) {
-        comparators[i].copy(bottom.slot, parentDoc);
-      }
-      if (!trackMaxScore && trackScores) {
-        score = scorer.score();
-      }
-      bottom.doc = docBase + parentDoc;
-      bottom.readerContext = currentReaderContext;
-      bottom.score = score;
-      copyGroups(bottom);
-      bottom = queue.updateTop();
+          //System.out.println("    competes!  doc=" + (docBase + parentDoc));
 
-      for (int i = 0; i < comparators.length; i++) {
-        comparators[i].setBottom(bottom.slot);
-      }
-    } else {
-      // Startup transient: queue is not yet full:
-      final int comparatorSlot = totalHitCount - 1;
+          // This hit is competitive - replace bottom element in queue & adjustTop
+          for (LeafFieldComparator comparator : comparators) {
+            comparator.copy(bottom.slot, parentDoc);
+          }
+          if (!trackMaxScore && trackScores) {
+            score = scorer.score();
+          }
+          bottom.doc = docBase + parentDoc;
+          bottom.readerContext = context;
+          bottom.score = score;
+          copyGroups(bottom);
+          bottom = queue.updateTop();
 
-      // Copy hit into queue
-      for (int i = 0; i < comparators.length; i++) {
-        comparators[i].copy(comparatorSlot, parentDoc);
-      }
-      //System.out.println("  startup: new OG doc=" + (docBase+parentDoc));
-      if (!trackMaxScore && trackScores) {
-        score = scorer.score();
-      }
-      final OneGroup og = new OneGroup(comparatorSlot, docBase+parentDoc, score, joinScorers.length, trackScores);
-      og.readerContext = currentReaderContext;
-      copyGroups(og);
-      bottom = queue.add(og);
-      queueFull = totalHitCount == numParentHits;
-      if (queueFull) {
-        // End of startup transient: queue just filled up:
-        for (int i = 0; i < comparators.length; i++) {
-          comparators[i].setBottom(bottom.slot);
+          for (LeafFieldComparator comparator : comparators) {
+            comparator.setBottom(bottom.slot);
+          }
+        } else {
+          // Startup transient: queue is not yet full:
+          final int comparatorSlot = totalHitCount - 1;
+
+          // Copy hit into queue
+          for (LeafFieldComparator comparator : comparators) {
+            comparator.copy(comparatorSlot, parentDoc);
+          }
+          //System.out.println("  startup: new OG doc=" + (docBase+parentDoc));
+          if (!trackMaxScore && trackScores) {
+            score = scorer.score();
+          }
+          final OneGroup og = new OneGroup(comparatorSlot, docBase+parentDoc, score, joinScorers.length, trackScores);
+          og.readerContext = context;
+          copyGroups(og);
+          bottom = queue.add(og);
+          queueFull = totalHitCount == numParentHits;
+          if (queueFull) {
+            // End of startup transient: queue just filled up:
+            for (LeafFieldComparator comparator : comparators) {
+              comparator.setBottom(bottom.slot);
+            }
+          }
         }
       }
-    }
-  }
-
-  // Pulls out child doc and scores for all join queries:
-  private void copyGroups(OneGroup og) {
-    // While rare, it's possible top arrays could be too
-    // short if join query had null scorer on first
-    // segment(s) but then became non-null on later segments
-    final int numSubScorers = joinScorers.length;
-    if (og.docs.length < numSubScorers) {
-      // While rare, this could happen if join query had
-      // null scorer on first segment(s) but then became
-      // non-null on later segments
-      og.docs = ArrayUtil.grow(og.docs);
-    }
-    if (og.counts.length < numSubScorers) {
-      og.counts = ArrayUtil.grow(og.counts);
-    }
-    if (trackScores && og.scores.length < numSubScorers) {
-      og.scores = ArrayUtil.grow(og.scores);
-    }
+      
+      // Pulls out child doc and scores for all join queries:
+      private void copyGroups(OneGroup og) {
+        // While rare, it's possible top arrays could be too
+        // short if join query had null scorer on first
+        // segment(s) but then became non-null on later segments
+        final int numSubScorers = joinScorers.length;
+        if (og.docs.length < numSubScorers) {
+          // While rare, this could happen if join query had
+          // null scorer on first segment(s) but then became
+          // non-null on later segments
+          og.docs = ArrayUtil.grow(og.docs);
+        }
+        if (og.counts.length < numSubScorers) {
+          og.counts = ArrayUtil.grow(og.counts);
+        }
+        if (trackScores && og.scores.length < numSubScorers) {
+          og.scores = ArrayUtil.grow(og.scores);
+        }
 
-    //System.out.println("\ncopyGroups parentDoc=" + og.doc);
-    for(int scorerIDX = 0;scorerIDX < numSubScorers;scorerIDX++) {
-      final ToParentBlockJoinQuery.BlockJoinScorer joinScorer = joinScorers[scorerIDX];
-      //System.out.println("  scorer=" + joinScorer);
-      if (joinScorer != null && docBase + joinScorer.getParentDoc() == og.doc) {
-        og.counts[scorerIDX] = joinScorer.getChildCount();
-        //System.out.println("    count=" + og.counts[scorerIDX]);
-        og.docs[scorerIDX] = joinScorer.swapChildDocs(og.docs[scorerIDX]);
-        assert og.docs[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.docs[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
-        //System.out.println("    len=" + og.docs[scorerIDX].length);
-        /*
-          for(int idx=0;idx<og.counts[scorerIDX];idx++) {
-          System.out.println("    docs[" + idx + "]=" + og.docs[scorerIDX][idx]);
+        //System.out.println("\ncopyGroups parentDoc=" + og.doc);
+        for(int scorerIDX = 0;scorerIDX < numSubScorers;scorerIDX++) {
+          final ToParentBlockJoinQuery.BlockJoinScorer joinScorer = joinScorers[scorerIDX];
+          //System.out.println("  scorer=" + joinScorer);
+          if (joinScorer != null && docBase + joinScorer.getParentDoc() == og.doc) {
+            og.counts[scorerIDX] = joinScorer.getChildCount();
+            //System.out.println("    count=" + og.counts[scorerIDX]);
+            og.docs[scorerIDX] = joinScorer.swapChildDocs(og.docs[scorerIDX]);
+            assert og.docs[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.docs[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
+            //System.out.println("    len=" + og.docs[scorerIDX].length);
+            /*
+              for(int idx=0;idx<og.counts[scorerIDX];idx++) {
+              System.out.println("    docs[" + idx + "]=" + og.docs[scorerIDX][idx]);
+              }
+            */
+            if (trackScores) {
+              //System.out.println("    copy scores");
+              og.scores[scorerIDX] = joinScorer.swapChildScores(og.scores[scorerIDX]);
+              assert og.scores[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.scores[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
+            }
+          } else {
+            og.counts[scorerIDX] = 0;
           }
-        */
-        if (trackScores) {
-          //System.out.println("    copy scores");
-          og.scores[scorerIDX] = joinScorer.swapChildScores(og.scores[scorerIDX]);
-          assert og.scores[scorerIDX].length >= og.counts[scorerIDX]: "length=" + og.scores[scorerIDX].length + " vs count=" + og.counts[scorerIDX];
         }
-      } else {
-        og.counts[scorerIDX] = 0;
       }
-    }
-  }
-
-  @Override
-  protected void doSetNextReader(LeafReaderContext context) throws IOException {
-    currentReaderContext = context;
-    docBase = context.docBase;
-    for (int compIDX = 0; compIDX < comparators.length; compIDX++) {
-      queue.setComparator(compIDX, comparators[compIDX].setNextReader(context));
-    }
-  }
-
-  @Override
-  public boolean acceptsDocsOutOfOrder() {
-    return false;
+    };
   }
 
   private void enroll(ToParentBlockJoinQuery query, ToParentBlockJoinQuery.BlockJoinScorer scorer) {
@@ -296,34 +316,6 @@ public class ToParentBlockJoinCollector
       joinScorers[slot] = scorer;
     }
   }
-  
-  @Override
-  public void setScorer(Scorer scorer) {
-    //System.out.println("C.setScorer scorer=" + scorer);
-    // Since we invoke .score(), and the comparators likely
-    // do as well, cache it so it's only "really" computed
-    // once:
-    this.scorer = new ScoreCachingWrappingScorer(scorer);
-    for (int compIDX = 0; compIDX < comparators.length; compIDX++) {
-      comparators[compIDX].setScorer(this.scorer);
-    }
-    Arrays.fill(joinScorers, null);
-
-    Queue<Scorer> queue = new LinkedList<>();
-    //System.out.println("\nqueue: add top scorer=" + scorer);
-    queue.add(scorer);
-    while ((scorer = queue.poll()) != null) {
-      //System.out.println("  poll: " + scorer + "; " + scorer.getWeight().getQuery());
-      if (scorer instanceof ToParentBlockJoinQuery.BlockJoinScorer) {
-        enroll((ToParentBlockJoinQuery) scorer.getWeight().getQuery(), (ToParentBlockJoinQuery.BlockJoinScorer) scorer);
-      }
-
-      for (ChildScorer sub : scorer.getChildren()) {
-        //System.out.println("  add sub: " + sub.child + "; " + sub.child.getWeight().getQuery());
-        queue.add(sub.child);
-      }
-    }
-  }
 
   private OneGroup[] sortedGroups;
 
@@ -414,14 +406,14 @@ public class ToParentBlockJoinCollector
         if (!trackScores) {
           throw new IllegalArgumentException("cannot sort by relevance within group: trackScores=false");
         }
-        collector = TopScoreDocCollector.create(numDocsInGroup, true);
+        collector = TopScoreDocCollector.create(numDocsInGroup);
       } else {
         // Sort by fields
-        collector = TopFieldCollector.create(withinGroupSort, numDocsInGroup, fillSortFields, trackScores, trackMaxScore, true);
+        collector = TopFieldCollector.create(withinGroupSort, numDocsInGroup, fillSortFields, trackScores, trackMaxScore);
       }
 
-      collector.setScorer(fakeScorer);
-      collector.getLeafCollector(og.readerContext);
+      LeafCollector leafCollector = collector.getLeafCollector(og.readerContext);
+      leafCollector.setScorer(fakeScorer);
       for(int docIDX=0;docIDX<numChildDocs;docIDX++) {
         //System.out.println("docIDX=" + docIDX + " vs " + og.docs[slot].length);
         final int doc = og.docs[slot][docIDX];
@@ -429,7 +421,7 @@ public class ToParentBlockJoinCollector
         if (trackScores) {
           fakeScorer.score = og.scores[slot][docIDX];
         }
-        collector.collect(doc);
+        leafCollector.collect(doc);
       }
       totalGroupedHitCount += numChildDocs;
 

Modified: lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinFieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinFieldComparator.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinFieldComparator.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinFieldComparator.java Sun Feb  8 23:53:14 2015
@@ -22,6 +22,8 @@ import java.io.IOException;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.SimpleFieldComparator;
 import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.BitSet;
 
@@ -31,13 +33,14 @@ import org.apache.lucene.util.BitSet;
  *
  * @lucene.experimental
  */
-public abstract class ToParentBlockJoinFieldComparator extends FieldComparator<Object> {
+public abstract class ToParentBlockJoinFieldComparator extends SimpleFieldComparator<Object> implements LeafFieldComparator { // repeat LeafFieldComparator for javadocs
 
   private final BitDocIdSetFilter parentFilter;
   private final BitDocIdSetFilter childFilter;
   final int spareSlot;
 
   FieldComparator<Object> wrappedComparator;
+  LeafFieldComparator wrappedLeafComparator;
   BitSet parentDocuments;
   BitSet childDocuments;
 
@@ -55,7 +58,7 @@ public abstract class ToParentBlockJoinF
 
   @Override
   public void setBottom(int slot) {
-    wrappedComparator.setBottom(slot);
+    wrappedLeafComparator.setBottom(slot);
   }
 
   @Override
@@ -64,7 +67,7 @@ public abstract class ToParentBlockJoinF
   }
 
   @Override
-  public FieldComparator<Object> setNextReader(LeafReaderContext context) throws IOException {
+  protected void doSetNextReader(LeafReaderContext context) throws IOException {
     BitDocIdSet children = childFilter.getDocIdSet(context);
     if (children == null) {
       childDocuments = null;
@@ -77,8 +80,7 @@ public abstract class ToParentBlockJoinF
     } else {
       parentDocuments = parents.bits();
     }
-    wrappedComparator = wrappedComparator.setNextReader(context);
-    return this;
+    wrappedLeafComparator = wrappedComparator.getLeafComparator(context);
   }
 
   @Override
@@ -90,12 +92,12 @@ public abstract class ToParentBlockJoinF
    * Concrete implementation of {@link ToParentBlockJoinSortField} to sorts the parent docs with the lowest values
    * in the child / nested docs first.
    */
-  public static final class Lowest extends ToParentBlockJoinFieldComparator {
+  public static final class Lowest extends ToParentBlockJoinFieldComparator implements LeafFieldComparator {
 
     /**
      * Create ToParentBlockJoinFieldComparator.Lowest
      *
-     * @param wrappedComparator The {@link FieldComparator} on the child / nested level.
+     * @param wrappedComparator The {@link LeafFieldComparator} on the child / nested level.
      * @param parentFilter Filter that identifies the parent documents.
      * @param childFilter Filter that defines which child / nested documents participates in sorting.
      * @param spareSlot The extra slot inside the wrapped comparator that is used to compare which nested document
@@ -119,7 +121,7 @@ public abstract class ToParentBlockJoinF
       }
 
       // We only need to emit a single cmp value for any matching child doc
-      int cmp = wrappedComparator.compareBottom(childDoc);
+      int cmp = wrappedLeafComparator.compareBottom(childDoc);
       if (cmp > 0) {
         return cmp;
       }
@@ -129,7 +131,7 @@ public abstract class ToParentBlockJoinF
         if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
           return cmp;
         }
-        int cmp1 = wrappedComparator.compareBottom(childDoc);
+        int cmp1 = wrappedLeafComparator.compareBottom(childDoc);
         if (cmp1 > 0) {
           return cmp1;
         } else {
@@ -152,23 +154,22 @@ public abstract class ToParentBlockJoinF
       if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
         return;
       }
-      wrappedComparator.copy(spareSlot, childDoc);
-      wrappedComparator.copy(slot, childDoc);
+      wrappedLeafComparator.copy(spareSlot, childDoc);
+      wrappedLeafComparator.copy(slot, childDoc);
 
       while (true) {
         childDoc = childDocuments.nextSetBit(childDoc + 1);
         if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
           return;
         }
-        wrappedComparator.copy(spareSlot, childDoc);
+        wrappedLeafComparator.copy(spareSlot, childDoc);
         if (wrappedComparator.compare(spareSlot, slot) < 0) {
-          wrappedComparator.copy(slot, childDoc);
+          wrappedLeafComparator.copy(slot, childDoc);
         }
       }
     }
 
     @Override
-    @SuppressWarnings("unchecked")
     public int compareTop(int parentDoc) throws IOException {
       if (parentDoc == 0 || parentDocuments == null || childDocuments == null) {
         return 0;
@@ -182,7 +183,7 @@ public abstract class ToParentBlockJoinF
       }
 
       // We only need to emit a single cmp value for any matching child doc
-      int cmp = wrappedComparator.compareBottom(childDoc);
+      int cmp = wrappedLeafComparator.compareBottom(childDoc);
       if (cmp > 0) {
         return cmp;
       }
@@ -192,7 +193,7 @@ public abstract class ToParentBlockJoinF
         if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
           return cmp;
         }
-        int cmp1 = wrappedComparator.compareTop(childDoc);
+        int cmp1 = wrappedLeafComparator.compareTop(childDoc);
         if (cmp1 > 0) {
           return cmp1;
         } else {
@@ -209,12 +210,12 @@ public abstract class ToParentBlockJoinF
    * Concrete implementation of {@link ToParentBlockJoinSortField} to sorts the parent docs with the highest values
    * in the child / nested docs first.
    */
-  public static final class Highest extends ToParentBlockJoinFieldComparator {
+  public static final class Highest extends ToParentBlockJoinFieldComparator implements LeafFieldComparator {
 
     /**
      * Create ToParentBlockJoinFieldComparator.Highest
      *
-     * @param wrappedComparator The {@link FieldComparator} on the child / nested level.
+     * @param wrappedComparator The {@link LeafFieldComparator} on the child / nested level.
      * @param parentFilter Filter that identifies the parent documents.
      * @param childFilter Filter that defines which child / nested documents participates in sorting.
      * @param spareSlot The extra slot inside the wrapped comparator that is used to compare which nested document
@@ -236,7 +237,7 @@ public abstract class ToParentBlockJoinF
         return 0;
       }
 
-      int cmp = wrappedComparator.compareBottom(childDoc);
+      int cmp = wrappedLeafComparator.compareBottom(childDoc);
       if (cmp < 0) {
         return cmp;
       }
@@ -246,7 +247,7 @@ public abstract class ToParentBlockJoinF
         if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
           return cmp;
         }
-        int cmp1 = wrappedComparator.compareBottom(childDoc);
+        int cmp1 = wrappedLeafComparator.compareBottom(childDoc);
         if (cmp1 < 0) {
           return cmp1;
         } else {
@@ -268,23 +269,22 @@ public abstract class ToParentBlockJoinF
       if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
         return;
       }
-      wrappedComparator.copy(spareSlot, childDoc);
-      wrappedComparator.copy(slot, childDoc);
+      wrappedLeafComparator.copy(spareSlot, childDoc);
+      wrappedLeafComparator.copy(slot, childDoc);
 
       while (true) {
         childDoc = childDocuments.nextSetBit(childDoc + 1);
         if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
           return;
         }
-        wrappedComparator.copy(spareSlot, childDoc);
+        wrappedLeafComparator.copy(spareSlot, childDoc);
         if (wrappedComparator.compare(spareSlot, slot) > 0) {
-          wrappedComparator.copy(slot, childDoc);
+          wrappedLeafComparator.copy(slot, childDoc);
         }
       }
     }
 
     @Override
-    @SuppressWarnings("unchecked")
     public int compareTop(int parentDoc) throws IOException {
       if (parentDoc == 0 || parentDocuments == null || childDocuments == null) {
         return 0;
@@ -296,7 +296,7 @@ public abstract class ToParentBlockJoinF
         return 0;
       }
 
-      int cmp = wrappedComparator.compareBottom(childDoc);
+      int cmp = wrappedLeafComparator.compareBottom(childDoc);
       if (cmp < 0) {
         return cmp;
       }
@@ -306,7 +306,7 @@ public abstract class ToParentBlockJoinF
         if (childDoc >= parentDoc || childDoc == DocIdSetIterator.NO_MORE_DOCS) {
           return cmp;
         }
-        int cmp1 = wrappedComparator.compareTop(childDoc);
+        int cmp1 = wrappedLeafComparator.compareTop(childDoc);
         if (cmp1 < 0) {
           return cmp1;
         } else {

Modified: lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java Sun Feb  8 23:53:14 2015
@@ -194,11 +194,6 @@ public class ToParentBlockJoinQuery exte
       }
       return new ComplexExplanation(false, 0.0f, "Not a match");
     }
-
-    @Override
-    public boolean scoresDocsOutOfOrder() {
-      return false;
-    }
   }
 
   static class BlockJoinScorer extends Scorer {

Modified: lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java Sun Feb  8 23:53:14 2015
@@ -18,6 +18,7 @@ package org.apache.lucene.search.join;
  */
 
 import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.SortField;
 
 import java.io.IOException;

Modified: lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java Sun Feb  8 23:53:14 2015
@@ -1323,6 +1323,7 @@ public class TestBlockJoin extends Lucen
     IndexReader r = w.getReader();
     w.close();
 
+    IndexSearcher searcher = new ToParentBlockJoinIndexSearcher(r);
     Query childQuery = new TermQuery(new Term("childText", "text"));
     BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
     ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
@@ -1332,7 +1333,7 @@ public class TestBlockJoin extends Lucen
 
     ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(new Sort(new SortField("parentID", SortField.Type.STRING)),
                                                                   10, true, true);
-    newSearcher(r).search(parentQuery, c);
+    searcher.search(parentQuery, c);
     TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false);
 
     // Two parents:
@@ -1388,6 +1389,8 @@ public class TestBlockJoin extends Lucen
     IndexReader r = w.getReader();
     w.close();
 
+    IndexSearcher searcher = new ToParentBlockJoinIndexSearcher(r);
+    
     // never matches:
     Query childQuery = new TermQuery(new Term("childText", "bogus"));
     BitDocIdSetFilter parentsFilter = new BitDocIdSetCachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new Term("isParent", "yes"))));
@@ -1398,7 +1401,7 @@ public class TestBlockJoin extends Lucen
 
     ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(new Sort(new SortField("parentID", SortField.Type.STRING)),
                                                                   10, true, true);
-    newSearcher(r).search(parentQuery, c);
+    searcher.search(parentQuery, c);
     TopGroups<Integer> groups = c.getTopGroups(childJoinQuery, null, 0, 10, 0, false);
 
     // Two parents:

Modified: lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (original)
+++ lucene/dev/branches/lucene6005/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java Sun Feb  8 23:53:14 2015
@@ -49,9 +49,12 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.FilterLeafCollector;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
@@ -291,10 +294,6 @@ public class TestJoinUtil extends Lucene
             assertFalse("optimized bulkScorer was not used for join query embedded in boolean query!", sawFive);
           }
         }
-        @Override
-        public boolean acceptsDocsOutOfOrder() {
-          return true;
-        }
       });
 
     indexSearcher.getIndexReader().close();
@@ -418,8 +417,7 @@ public class TestJoinUtil extends Lucene
           dir,
           newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy())
       );
-      final boolean scoreDocsInOrder = TestJoinUtil.random().nextBoolean();
-      IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, multipleValuesPerDocument, scoreDocsInOrder);
+      IndexIterationContext context = createContext(numberOfDocumentsToIndex, w, multipleValuesPerDocument);
 
       IndexReader topLevelReader = w.getReader();
       w.close();
@@ -455,31 +453,21 @@ public class TestJoinUtil extends Lucene
 
         // Need to know all documents that have matches. TopDocs doesn't give me that and then I'd be also testing TopDocsCollector...
         final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
-        final TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(10, false);
-        indexSearcher.search(joinQuery, new SimpleCollector() {
-
-          int docBase;
-
-          @Override
-          public void collect(int doc) throws IOException {
-            actualResult.set(doc + docBase);
-            topScoreDocCollector.collect(doc);
-          }
-
-          @Override
-          protected void doSetNextReader(LeafReaderContext context) throws IOException {
-            docBase = context.docBase;
-            topScoreDocCollector.getLeafCollector(context);
-          }
+        final TopScoreDocCollector topScoreDocCollector = TopScoreDocCollector.create(10);
+        indexSearcher.search(joinQuery, new Collector() {
 
           @Override
-          public void setScorer(Scorer scorer) throws IOException {
-            topScoreDocCollector.setScorer(scorer);
-          }
-
-          @Override
-          public boolean acceptsDocsOutOfOrder() {
-            return scoreDocsInOrder;
+          public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+            final int docBase = context.docBase;
+            final LeafCollector in = topScoreDocCollector.getLeafCollector(context);
+            return new FilterLeafCollector(in) {
+
+              @Override
+              public void collect(int doc) throws IOException {
+                super.collect(doc);
+                actualResult.set(doc + docBase);
+              }
+            };
           }
         });
         // Asserting bit set...
@@ -523,11 +511,11 @@ public class TestJoinUtil extends Lucene
     }
   }
 
-  private IndexIterationContext createContext(int nDocs, RandomIndexWriter writer, boolean multipleValuesPerDocument, boolean scoreDocsInOrder) throws IOException {
-    return createContext(nDocs, writer, writer, multipleValuesPerDocument, scoreDocsInOrder);
+  private IndexIterationContext createContext(int nDocs, RandomIndexWriter writer, boolean multipleValuesPerDocument) throws IOException {
+    return createContext(nDocs, writer, writer, multipleValuesPerDocument);
   }
 
-  private IndexIterationContext createContext(int nDocs, RandomIndexWriter fromWriter, RandomIndexWriter toWriter, boolean multipleValuesPerDocument, boolean scoreDocsInOrder) throws IOException {
+  private IndexIterationContext createContext(int nDocs, RandomIndexWriter fromWriter, RandomIndexWriter toWriter, boolean multipleValuesPerDocument) throws IOException {
     IndexIterationContext context = new IndexIterationContext();
     int numRandomValues = nDocs / 2;
     context.randomUniqueValues = new String[numRandomValues];
@@ -655,11 +643,6 @@ public class TestJoinUtil extends Lucene
           public void setScorer(Scorer scorer) {
             this.scorer = scorer;
           }
-
-          @Override
-          public boolean acceptsDocsOutOfOrder() {
-            return false;
-          }
         });
       } else {
         fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() {
@@ -692,76 +675,33 @@ public class TestJoinUtil extends Lucene
           public void setScorer(Scorer scorer) {
             this.scorer = scorer;
           }
-
-          @Override
-          public boolean acceptsDocsOutOfOrder() {
-            return false;
-          }
         });
       }
 
       final Map<Integer, JoinScore> docToJoinScore = new HashMap<>();
       if (multipleValuesPerDocument) {
-        if (scoreDocsInOrder) {
-          LeafReader slowCompositeReader = SlowCompositeReaderWrapper.wrap(toSearcher.getIndexReader());
-          Terms terms = slowCompositeReader.terms(toField);
-          if (terms != null) {
-            DocsEnum docsEnum = null;
-            TermsEnum termsEnum = null;
-            SortedSet<BytesRef> joinValues = new TreeSet<>(BytesRef.getUTF8SortedAsUnicodeComparator());
-            joinValues.addAll(joinValueToJoinScores.keySet());
-            for (BytesRef joinValue : joinValues) {
-              termsEnum = terms.iterator(termsEnum);
-              if (termsEnum.seekExact(joinValue)) {
-                docsEnum = termsEnum.docs(slowCompositeReader.getLiveDocs(), docsEnum, DocsEnum.FLAG_NONE);
-                JoinScore joinScore = joinValueToJoinScores.get(joinValue);
-
-                for (int doc = docsEnum.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = docsEnum.nextDoc()) {
-                  // First encountered join value determines the score.
-                  // Something to keep in mind for many-to-many relations.
-                  if (!docToJoinScore.containsKey(doc)) {
-                    docToJoinScore.put(doc, joinScore);
-                  }
-                }
-              }
-            }
-          }
-        } else {
-          toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
-
-            private SortedSetDocValues docTermOrds;
-            private int docBase;
+        LeafReader slowCompositeReader = SlowCompositeReaderWrapper.wrap(toSearcher.getIndexReader());
+        Terms terms = slowCompositeReader.terms(toField);
+        if (terms != null) {
+          DocsEnum docsEnum = null;
+          TermsEnum termsEnum = null;
+          SortedSet<BytesRef> joinValues = new TreeSet<>(BytesRef.getUTF8SortedAsUnicodeComparator());
+          joinValues.addAll(joinValueToJoinScores.keySet());
+          for (BytesRef joinValue : joinValues) {
+            termsEnum = terms.iterator(termsEnum);
+            if (termsEnum.seekExact(joinValue)) {
+              docsEnum = termsEnum.docs(slowCompositeReader.getLiveDocs(), docsEnum, DocsEnum.FLAG_NONE);
+              JoinScore joinScore = joinValueToJoinScores.get(joinValue);
 
-            @Override
-            public void collect(int doc) throws IOException {
-              docTermOrds.setDocument(doc);
-              long ord;
-              while ((ord = docTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-                final BytesRef joinValue = docTermOrds.lookupOrd(ord);
-                JoinScore joinScore = joinValueToJoinScores.get(joinValue);
-                if (joinScore == null) {
-                  continue;
-                }
-                Integer basedDoc = docBase + doc;
+              for (int doc = docsEnum.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = docsEnum.nextDoc()) {
                 // First encountered join value determines the score.
                 // Something to keep in mind for many-to-many relations.
-                if (!docToJoinScore.containsKey(basedDoc)) {
-                  docToJoinScore.put(basedDoc, joinScore);
+                if (!docToJoinScore.containsKey(doc)) {
+                  docToJoinScore.put(doc, joinScore);
                 }
               }
             }
-
-            @Override
-            protected void doSetNextReader(LeafReaderContext context) throws IOException {
-              docBase = context.docBase;
-              docTermOrds = DocValues.getSortedSet(context.reader(), toField);
-            }
-
-            @Override
-            public boolean acceptsDocsOutOfOrder() {return false;}
-            @Override
-            public void setScorer(Scorer scorer) {}
-          });
+          }
         }
       } else {
         toSearcher.search(new MatchAllDocsQuery(), new SimpleCollector() {
@@ -786,8 +726,6 @@ public class TestJoinUtil extends Lucene
           }
 
           @Override
-          public boolean acceptsDocsOutOfOrder() {return false;}
-          @Override
           public void setScorer(Scorer scorer) {}
         });
       }

Modified: lucene/dev/branches/lucene6005/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/branches/lucene6005/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Sun Feb  8 23:53:14 2015
@@ -29,6 +29,7 @@ import org.apache.lucene.analysis.Analyz
 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.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.document.FieldTypes;
@@ -61,6 +62,8 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefArray;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
@@ -188,17 +191,19 @@ import org.apache.lucene.util.RecyclingI
  */
 public class MemoryIndex {
 
+  private static final boolean DEBUG = false;
+
   /** info for each field: Map&lt;String fieldName, Info field&gt; */
   private final SortedMap<String,Info> fields = new TreeMap<>();
   
   private final boolean storeOffsets;
-  
-  private static final boolean DEBUG = false;
+  private final boolean storePayloads;
 
   private final ByteBlockPool byteBlockPool;
   private final IntBlockPool intBlockPool;
 //  private final IntBlockPool.SliceReader postingsReader;
   private final IntBlockPool.SliceWriter postingsWriter;
+  private final BytesRefArray payloadsBytesRefs;//non null only when storePayloads
 
   private Counter bytesUsed;
 
@@ -207,7 +212,7 @@ public class MemoryIndex {
   private Similarity normSimilarity = IndexSearcher.getDefaultSimilarity();
 
   /**
-   * Constructs an empty instance.
+   * Constructs an empty instance that will not store offsets or payloads.
    */
   public MemoryIndex() {
     this(false);
@@ -216,25 +221,37 @@ public class MemoryIndex {
   /**
    * Constructs an empty instance that can optionally store the start and end
    * character offset of each token term in the text. This can be useful for
-   * highlighting of hit locations with the Lucene highlighter package.
-   * Protected until the highlighter package matures, so that this can actually
-   * be meaningfully integrated.
+   * highlighting of hit locations with the Lucene highlighter package.  But
+   * it will not store payloads; use another constructor for that.
    * 
    * @param storeOffsets
    *            whether or not to store the start and end character offset of
    *            each token term in the text
    */
   public MemoryIndex(boolean storeOffsets) {
-    this(storeOffsets, 0);
+    this(storeOffsets, false);
   }
-  
+
+  /**
+   * Constructs an empty instance with the option of storing offsets and payloads.
+   *
+   * @param storeOffsets store term offsets at each position
+   * @param storePayloads store term payloads at each position
+   */
+  public MemoryIndex(boolean storeOffsets, boolean storePayloads) {
+    this(storeOffsets, storePayloads, 0);
+  }
+
   /**
    * Expert: This constructor accepts an upper limit for the number of bytes that should be reused if this instance is {@link #reset()}.
+   * The payload storage, if used, is unaffected by maxReusuedBytes, however.
    * @param storeOffsets <code>true</code> if offsets should be stored
+   * @param storePayloads <code>true</code> if payloads should be stored
    * @param maxReusedBytes the number of bytes that should remain in the internal memory pools after {@link #reset()} is called
    */
-  MemoryIndex(boolean storeOffsets, long maxReusedBytes) {
+  MemoryIndex(boolean storeOffsets, boolean storePayloads, long maxReusedBytes) {
     this.storeOffsets = storeOffsets;
+    this.storePayloads = storePayloads;
     this.bytesUsed = Counter.newCounter();
     final int maxBufferedByteBlocks = (int)((maxReusedBytes/2) / ByteBlockPool.BYTE_BLOCK_SIZE );
     final int maxBufferedIntBlocks = (int) ((maxReusedBytes - (maxBufferedByteBlocks*ByteBlockPool.BYTE_BLOCK_SIZE))/(IntBlockPool.INT_BLOCK_SIZE * RamUsageEstimator.NUM_BYTES_INT));
@@ -242,6 +259,8 @@ public class MemoryIndex {
     byteBlockPool = new ByteBlockPool(new RecyclingByteBlockAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, maxBufferedByteBlocks, bytesUsed));
     intBlockPool = new IntBlockPool(new RecyclingIntBlockAllocator(IntBlockPool.INT_BLOCK_SIZE, maxBufferedIntBlocks, bytesUsed));
     postingsWriter = new SliceWriter(intBlockPool);
+    //TODO refactor BytesRefArray to allow us to apply maxReusedBytes option
+    payloadsBytesRefs = storePayloads ? new BytesRefArray(bytesUsed) : null;
   }
   
   /**
@@ -382,8 +401,8 @@ public class MemoryIndex {
    *
    * @param fieldName
    *            a name to be associated with the text
-   * @param stream
-   *            the token stream to retrieve tokens from.
+   * @param tokenStream
+   *            the token stream to retrieve tokens from. It's guaranteed to be closed no matter what.
    * @param boost
    *            the boost factor for hits for this field
    * @param positionIncrementGap
@@ -392,16 +411,17 @@ public class MemoryIndex {
    *            the offset gap if fields with the same name are added more than once
    * @see org.apache.lucene.document.Field#setBoost(float)
    */
-  public void addField(String fieldName, TokenStream stream, float boost, int positionIncrementGap, int offsetGap) {
-    try {
+  public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap,
+                       int offsetGap) {
+    try (TokenStream stream = tokenStream) {
       if (frozen)
         throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
       if (fieldName == null)
         throw new IllegalArgumentException("fieldName must not be null");
       if (stream == null)
-          throw new IllegalArgumentException("token stream must not be null");
+        throw new IllegalArgumentException("token stream must not be null");
       if (boost <= 0.0f)
-          throw new IllegalArgumentException("boost factor must be greater than 0.0");
+        throw new IllegalArgumentException("boost factor must be greater than 0.0");
       int numTokens = 0;
       int numOverlapTokens = 0;
       int pos = -1;
@@ -422,8 +442,9 @@ public class MemoryIndex {
         sliceArray = info.sliceArray;
         sumTotalTermFreq = info.sumTotalTermFreq;
       } else {
-        fieldInfo = new FieldInfo(fieldName, fields.size(), false, false, false,
-            this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS,
+        fieldInfo = new FieldInfo(fieldName, fields.size(), false, false, this.storePayloads,
+            this.storeOffsets
+                ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS,
             DocValuesType.NONE, -1, null);
         sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
         terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);
@@ -432,6 +453,7 @@ public class MemoryIndex {
       TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
       PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
       OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
+      PayloadAttribute payloadAtt = storePayloads ? stream.addAttribute(PayloadAttribute.class) : null;
       BytesRef ref = termAtt.getBytesRef();
       stream.reset();
       
@@ -452,13 +474,16 @@ public class MemoryIndex {
         }
         sliceArray.freq[ord]++;
         sumTotalTermFreq++;
-        if (!storeOffsets) {
-          postingsWriter.writeInt(pos);
-        } else {
-          postingsWriter.writeInt(pos);
+        postingsWriter.writeInt(pos);
+        if (storeOffsets) {
           postingsWriter.writeInt(offsetAtt.startOffset() + offset);
           postingsWriter.writeInt(offsetAtt.endOffset() + offset);
         }
+        if (storePayloads) {
+          final BytesRef payload = payloadAtt.getPayload();
+          int pIndex = payload == null ? -1 : payloadsBytesRefs.append(payload);
+          postingsWriter.writeInt(pIndex);
+        }
         sliceArray.end[ord] = postingsWriter.getCurrentOffset();
       }
       stream.end();
@@ -467,16 +492,8 @@ public class MemoryIndex {
       if (numTokens > 0) {
         fields.put(fieldName, new Info(fieldInfo, terms, sliceArray, numTokens, numOverlapTokens, boost, pos, offsetAtt.endOffset() + offset, sumTotalTermFreq));
       }
-    } catch (Exception e) { // can never happen
+    } catch (IOException e) {
       throw new RuntimeException(e);
-    } finally {
-      try {
-        if (stream != null) {
-          stream.close();
-        }
-      } catch (IOException e2) {
-        throw new RuntimeException(e2);
-      }
     }
   }
 
@@ -553,11 +570,6 @@ public class MemoryIndex {
           this.scorer = scorer;
         }
 
-        @Override
-        public boolean acceptsDocsOutOfOrder() {
-          return true;
-        }
-
       });
       float score = scores[0];
       return score;
@@ -869,7 +881,7 @@ public class MemoryIndex {
 
           @Override
           public boolean hasPayloads() {
-            return false;
+            return storePayloads;
           }
         };
       }
@@ -1052,17 +1064,20 @@ public class MemoryIndex {
     }
     
     private class MemoryDocsAndPositionsEnum extends DocsAndPositionsEnum {
+      private final SliceReader sliceReader;
       private int posUpto; // for assert
       private boolean hasNext;
       private Bits liveDocs;
       private int doc = -1;
-      private SliceReader sliceReader;
       private int freq;
       private int startOffset;
       private int endOffset;
-      
+      private int payloadIndex;
+      private final BytesRefBuilder payloadBuilder;//only non-null when storePayloads
+
       public MemoryDocsAndPositionsEnum() {
         this.sliceReader = new SliceReader(intBlockPool);
+        this.payloadBuilder = storePayloads ? new BytesRefBuilder() : null;
       }
 
       public DocsAndPositionsEnum reset(Bits liveDocs, int start, int end, int freq) {
@@ -1105,14 +1120,15 @@ public class MemoryIndex {
       public int nextPosition() {
         assert posUpto++ < freq;
         assert !sliceReader.endOfSlice() : " stores offsets : " + startOffset;
+        int pos = sliceReader.readInt();
         if (storeOffsets) {
-          int pos = sliceReader.readInt();
           startOffset = sliceReader.readInt();
           endOffset = sliceReader.readInt();
-          return pos;
-        } else {
-          return sliceReader.readInt();
         }
+        if (storePayloads) {
+          payloadIndex = sliceReader.readInt();
+        }
+        return pos;
       }
 
       @Override
@@ -1127,7 +1143,10 @@ public class MemoryIndex {
 
       @Override
       public BytesRef getPayload() {
-        return null;
+        if (payloadBuilder == null || payloadIndex == -1) {
+          return null;
+        }
+        return payloadsBytesRefs.get(payloadBuilder, payloadIndex);
       }
       
       @Override
@@ -1187,6 +1206,9 @@ public class MemoryIndex {
     this.normSimilarity = IndexSearcher.getDefaultSimilarity();
     byteBlockPool.reset(false, false); // no need to 0-fill the buffers
     intBlockPool.reset(true, false); // here must must 0-fill since we use slices
+    if (payloadsBytesRefs != null) {
+      payloadsBytesRefs.clear();
+    }
     this.frozen = false;
   }
   

Modified: lucene/dev/branches/lucene6005/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java (original)
+++ lucene/dev/branches/lucene6005/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java Sun Feb  8 23:53:14 2015
@@ -67,8 +67,8 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
-import org.apache.lucene.util.ByteBlockPool.Allocator;
 import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.ByteBlockPool.Allocator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LineFileDocs;
@@ -115,7 +115,7 @@ public class TestMemoryIndexAgainstRAMDi
    * runs random tests, up to ITERATIONS times.
    */
   public void testRandomQueries() throws Exception {
-    MemoryIndex index =  new MemoryIndex(random().nextBoolean(), random().nextInt(50) * 1024 * 1024);
+    MemoryIndex index = randomMemoryIndex();
     for (int i = 0; i < ITERATIONS; i++) {
       assertAgainstRAMDirectory(index);
     }
@@ -147,7 +147,8 @@ public class TestMemoryIndexAgainstRAMDi
     Directory ramdir = new RAMDirectory();
     Analyzer analyzer = randomAnalyzer();
     IndexWriter writer = new IndexWriter(ramdir,
-                                         new IndexWriterConfig(analyzer).setCodec(TestUtil.alwaysPostingsFormat(TestUtil.getDefaultPostingsFormat())));
+                                         new IndexWriterConfig(analyzer).setCodec(
+                                             TestUtil.alwaysPostingsFormat(TestUtil.getDefaultPostingsFormat())));
     Document doc = writer.newDocument();
     doc.addLargeText("foo", fooField.toString());
     doc.addLargeText("term", termField.toString());
@@ -206,6 +207,10 @@ public class TestMemoryIndexAgainstRAMDi
                   assertEquals(iwDocsAndPos.startOffset(), memDocsAndPos.startOffset());
                   assertEquals(iwDocsAndPos.endOffset(), memDocsAndPos.endOffset());
                 }
+
+                if (iwTerms.hasPayloads()) {
+                  assertEquals(iwDocsAndPos.getPayload(), memDocsAndPos.getPayload());
+                }
               }
               
             }
@@ -308,7 +313,7 @@ public class TestMemoryIndexAgainstRAMDi
   
   public void testDocsEnumStart() throws Exception {
     Analyzer analyzer = new MockAnalyzer(random());
-    MemoryIndex memory = new MemoryIndex(random().nextBoolean(),  random().nextInt(50) * 1024 * 1024);
+    MemoryIndex memory = new MemoryIndex(random().nextBoolean(), false, random().nextInt(50) * 1024 * 1024);
     memory.addField("foo", "bar", analyzer);
     LeafReader reader = (LeafReader) memory.createSearcher().getIndexReader();
     DocsEnum disi = TestUtil.docs(random(), reader, "foo", new BytesRef("bar"), null, null, DocsEnum.FLAG_NONE);
@@ -333,11 +338,15 @@ public class TestMemoryIndexAgainstRAMDi
       return new ByteBlockPool.DirectAllocator();
     }
   }
-  
+
+  private MemoryIndex randomMemoryIndex() {
+    return new MemoryIndex(random().nextBoolean(), random().nextBoolean(), random().nextInt(50) * 1024 * 1024);
+  }
+
   public void testDocsAndPositionsEnumStart() throws Exception {
     Analyzer analyzer = new MockAnalyzer(random());
     int numIters = atLeast(3);
-    MemoryIndex memory = new MemoryIndex(true,  random().nextInt(50) * 1024 * 1024);
+    MemoryIndex memory = new MemoryIndex(true, false, random().nextInt(50) * 1024 * 1024);
     for (int i = 0; i < numIters; i++) { // check reuse
       memory.addField("foo", "bar", analyzer);
       LeafReader reader = (LeafReader) memory.createSearcher().getIndexReader();
@@ -367,7 +376,7 @@ public class TestMemoryIndexAgainstRAMDi
     RegexpQuery regex = new RegexpQuery(new Term("field", "worl."));
     SpanQuery wrappedquery = new SpanMultiTermQueryWrapper<>(regex);
         
-    MemoryIndex mindex = new MemoryIndex(random().nextBoolean(),  random().nextInt(50) * 1024 * 1024);
+    MemoryIndex mindex = randomMemoryIndex();
     mindex.addField("field", new MockAnalyzer(random()).tokenStream("field", "hello there"));
 
     // This throws an NPE
@@ -379,7 +388,7 @@ public class TestMemoryIndexAgainstRAMDi
     RegexpQuery regex = new RegexpQuery(new Term("field", "worl."));
     SpanQuery wrappedquery = new SpanOrQuery(new SpanMultiTermQueryWrapper<>(regex));
 
-    MemoryIndex mindex = new MemoryIndex(random().nextBoolean(),  random().nextInt(50) * 1024 * 1024);
+    MemoryIndex mindex = randomMemoryIndex();
     mindex.addField("field", new MockAnalyzer(random()).tokenStream("field", "hello there"));
 
     // This passes though
@@ -387,7 +396,7 @@ public class TestMemoryIndexAgainstRAMDi
   }
   
   public void testSameFieldAddedMultipleTimes() throws IOException {
-    MemoryIndex mindex = new MemoryIndex(random().nextBoolean(),  random().nextInt(50) * 1024 * 1024);
+    MemoryIndex mindex = randomMemoryIndex();
     MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
     mindex.addField("field", "the quick brown fox", mockAnalyzer);
     mindex.addField("field", "jumps over the", mockAnalyzer);
@@ -406,8 +415,8 @@ public class TestMemoryIndexAgainstRAMDi
     assertTrue("posGap" + mockAnalyzer.getPositionIncrementGap("field") , mindex.search(query) > 0.0001);
   }
   
-  public void testNonExistingsField() throws IOException {
-    MemoryIndex mindex = new MemoryIndex(random().nextBoolean(),  random().nextInt(50) * 1024 * 1024);
+  public void testNonExistentField() throws IOException {
+    MemoryIndex mindex = randomMemoryIndex();
     MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
     mindex.addField("field", "the quick brown fox", mockAnalyzer);
     LeafReader reader = (LeafReader) mindex.createSearcher().getIndexReader();
@@ -417,10 +426,10 @@ public class TestMemoryIndexAgainstRAMDi
     assertNull(reader.termPositionsEnum(new Term("not-in-index", "foo")));
     assertNull(reader.terms("not-in-index"));
   }
-  
+
   public void testDuellMemIndex() throws IOException {
     int numDocs = atLeast(10);
-    MemoryIndex memory = new MemoryIndex(random().nextBoolean(),  random().nextInt(50) * 1024 * 1024);
+    MemoryIndex memory = randomMemoryIndex();
     for (int i = 0; i < numDocs; i++) {
       Directory dir = newDirectory();
       MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
@@ -532,7 +541,7 @@ public class TestMemoryIndexAgainstRAMDi
         assertThat("Position test failed" + failDesc, memPos, equalTo(pos));
         assertThat("Start offset test failed" + failDesc, memDocsPosEnum.startOffset(), equalTo(docsPosEnum.startOffset()));
         assertThat("End offset test failed" + failDesc, memDocsPosEnum.endOffset(), equalTo(docsPosEnum.endOffset()));
-        assertThat("Missing payload test failed" + failDesc, docsPosEnum.getPayload(), equalTo(null));
+        assertThat("Missing payload test failed" + failDesc, docsPosEnum.getPayload(), equalTo(docsPosEnum.getPayload()));
       }
     }
     assertNull("Still some tokens not processed", memTermEnum.next());

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MergeReaderWrapper.java Sun Feb  8 23:53:14 2015
@@ -39,7 +39,7 @@ class MergeReaderWrapper extends LeafRea
   MergeReaderWrapper(SegmentReader in) throws IOException {
     this.in = in;
     
-    FieldsProducer fields = in.fields();
+    FieldsProducer fields = in.getPostingsReader();
     if (fields != null) {
       fields = fields.getMergeInstance();
     }

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java Sun Feb  8 23:53:14 2015
@@ -33,7 +33,7 @@ import org.apache.lucene.util.Version;
 
 /**
  * This tool splits input index into multiple equal parts. The method employed
- * here uses {@link IndexWriter#addIndexes(IndexReader[])} where the input data
+ * here uses {@link IndexWriter#addIndexes(CodecReader[])} where the input data
  * comes from the input index with artificially applied deletes to the document
  * id-s that fall outside the selected partition.
  * <p>Note 1: Deletes are only applied to a buffered list of deleted docs and
@@ -59,7 +59,7 @@ public class MultiPassIndexSplitter {
    * assigned in a deterministic round-robin fashion to one of the output splits.
    * @throws IOException If there is a low-level I/O error
    */
-  public void split(Version version, IndexReader in, Directory[] outputs, boolean seq) throws IOException {
+  public void split(IndexReader in, Directory[] outputs, boolean seq) throws IOException {
     if (outputs == null || outputs.length < 2) {
       throw new IOException("Invalid number of outputs.");
     }
@@ -102,7 +102,7 @@ public class MultiPassIndexSplitter {
       System.err.println("Writing part " + (i + 1) + " ...");
       // pass the subreaders directly, as our wrapper's numDocs/hasDeletetions are not up-to-date
       final List<? extends FakeDeleteLeafIndexReader> sr = input.getSequentialSubReaders();
-      w.addIndexes(sr.toArray(new IndexReader[sr.size()])); // TODO: maybe take List<IR> here?
+      w.addIndexes(sr.toArray(new CodecReader[sr.size()])); // TODO: maybe take List<IR> here?
       w.close();
     }
     System.err.println("Done.");
@@ -170,7 +170,7 @@ public class MultiPassIndexSplitter {
     } else {
       input = new MultiReader(indexes.toArray(new IndexReader[indexes.size()]));
     }
-    splitter.split(Version.LATEST, input, dirs, seq);
+    splitter.split(input, dirs, seq);
   }
   
   /**
@@ -178,16 +178,16 @@ public class MultiPassIndexSplitter {
    */
   private static final class FakeDeleteIndexReader extends BaseCompositeReader<FakeDeleteLeafIndexReader> {
 
-    public FakeDeleteIndexReader(IndexReader reader) {
+    public FakeDeleteIndexReader(IndexReader reader) throws IOException {
       super(initSubReaders(reader));
     }
     
-    private static FakeDeleteLeafIndexReader[] initSubReaders(IndexReader reader) {
+    private static FakeDeleteLeafIndexReader[] initSubReaders(IndexReader reader) throws IOException {
       final List<LeafReaderContext> leaves = reader.leaves();
       final FakeDeleteLeafIndexReader[] subs = new FakeDeleteLeafIndexReader[leaves.size()];
       int i = 0;
       for (final LeafReaderContext ctx : leaves) {
-        subs[i++] = new FakeDeleteLeafIndexReader(ctx.reader());
+        subs[i++] = new FakeDeleteLeafIndexReader(SlowCodecReaderWrapper.wrap(ctx.reader()));
       }
       return subs;
     }
@@ -210,10 +210,10 @@ public class MultiPassIndexSplitter {
     // as we pass the subreaders directly to IW.addIndexes().
   }
   
-  private static final class FakeDeleteLeafIndexReader extends FilterLeafReader {
+  private static final class FakeDeleteLeafIndexReader extends FilterCodecReader {
     FixedBitSet liveDocs;
 
-    public FakeDeleteLeafIndexReader(LeafReader reader) {
+    public FakeDeleteLeafIndexReader(CodecReader reader) {
       super(reader);
       undeleteAll(); // initialize main bitset
     }

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/PKIndexSplitter.java Sun Feb  8 23:53:14 2015
@@ -98,12 +98,12 @@ public class PKIndexSplitter {
     }
   }
   
-  private void createIndex(IndexWriterConfig config, Directory target, IndexReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
+  private void createIndex(IndexWriterConfig config, Directory target, DirectoryReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
     boolean success = false;
     final IndexWriter w = new IndexWriter(target, config);
     try {
       final List<LeafReaderContext> leaves = reader.leaves();
-      final IndexReader[] subReaders = new IndexReader[leaves.size()];
+      final CodecReader[] subReaders = new CodecReader[leaves.size()];
       int i = 0;
       for (final LeafReaderContext ctx : leaves) {
         subReaders[i++] = new DocumentFilteredLeafIndexReader(ctx, preserveFilter, negateFilter);
@@ -119,12 +119,13 @@ public class PKIndexSplitter {
     }
   }
     
-  private static class DocumentFilteredLeafIndexReader extends FilterLeafReader {
+  private static class DocumentFilteredLeafIndexReader extends FilterCodecReader {
     final Bits liveDocs;
     final int numDocs;
     
     public DocumentFilteredLeafIndexReader(LeafReaderContext context, Filter preserveFilter, boolean negateFilter) throws IOException {
-      super(context.reader());
+      // our cast is ok, since we open the Directory.
+      super((CodecReader) context.reader());
       final int maxDoc = in.maxDoc();
       final FixedBitSet bits = new FixedBitSet(maxDoc);
       // ignore livedocs here, as we filter them later:

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/Sorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/Sorter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/Sorter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/Sorter.java Sun Feb  8 23:53:14 2015
@@ -20,9 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Comparator;
 
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
@@ -214,12 +212,11 @@ final class Sorter {
   DocMap sort(LeafReader reader) throws IOException {
     SortField fields[] = sort.getSort();
     final int reverseMul[] = new int[fields.length];
-    final FieldComparator<?> comparators[] = new FieldComparator[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);
-      comparators[i].setNextReader(reader.getContext());
+      comparators[i] = fields[i].getComparator(1, i).getLeafComparator(reader.getContext());
       comparators[i].setScorer(FAKESCORER);
     }
     final DocComparator comparator = new DocComparator() {

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/index/SortingMergePolicy.java Sun Feb  8 23:53:14 2015
@@ -47,7 +47,7 @@ import org.apache.lucene.util.packed.Pac
  *  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, Analyzer) IndexWriter.addDocuments}
+ *  {@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 
@@ -65,7 +65,7 @@ public final class SortingMergePolicy ex
   
   class SortingOneMerge extends OneMerge {
 
-    List<LeafReader> unsortedReaders;
+    List<CodecReader> unsortedReaders;
     Sorter.DocMap docMap;
     LeafReader sortedView;
     final InfoStream infoStream;
@@ -76,7 +76,7 @@ public final class SortingMergePolicy ex
     }
 
     @Override
-    public List<LeafReader> getMergeReaders() throws IOException {
+    public List<CodecReader> getMergeReaders() throws IOException {
       if (unsortedReaders == null) {
         unsortedReaders = super.getMergeReaders();
         if (infoStream.isEnabled("SMP")) {
@@ -117,7 +117,7 @@ public final class SortingMergePolicy ex
         if (infoStream.isEnabled("SMP")) {
           infoStream.message("SMP", "sorting readers by " + sort);
         }
-        return Collections.singletonList(sortedView);
+        return Collections.singletonList(SlowCodecReaderWrapper.wrap(sortedView));
       }
     }
     
@@ -128,7 +128,7 @@ public final class SortingMergePolicy ex
       super.setInfo(info);
     }
 
-    private PackedLongValues getDeletes(List<LeafReader> readers) {
+    private PackedLongValues getDeletes(List<CodecReader> readers) {
       PackedLongValues.Builder deletes = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
       int deleteCount = 0;
       for (LeafReader reader : readers) {