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 2014/03/06 22:48:24 UTC

svn commit: r1575057 - in /lucene/dev/branches/lucene5487: lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/test/org/apache/lucene/search/ lucene/join/src/java/org/apache/lucene/search/join/ lucene/test-framework/src/java/org/apache/lucen...

Author: mikemccand
Date: Thu Mar  6 21:48:23 2014
New Revision: 1575057

URL: http://svn.apache.org/r1575057
Log:
LUCENE-5487: add TopScorers to FilteredQuery too; fix Solr; resolve all nocommits

Modified:
    lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
    lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
    lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/TopScorer.java
    lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/Weight.java
    lucene/dev/branches/lucene5487/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
    lucene/dev/branches/lucene5487/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
    lucene/dev/branches/lucene5487/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java
    lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/schema/LatLonType.java
    lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
    lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
    lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java

Modified: lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Thu Mar  6 21:48:23 2014
@@ -117,7 +117,7 @@ final class BooleanScorer extends TopSco
   private static final class FakeScorer extends Scorer {
 
     float score;
-    int doc;
+    int doc = -1;
     int freq;
     
     public FakeScorer() {
@@ -272,7 +272,7 @@ final class BooleanScorer extends TopSco
           // that should work)... but in theory an outside
           // app could pass a different max so we must check
           // it:
-          if (current.doc >= max){
+          if (current.doc >= max) {
             tmp = current;
             current = current.next;
             tmp.next = bucketTable.first;

Modified: lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Thu Mar  6 21:48:23 2014
@@ -124,10 +124,10 @@ public class FilteredQuery extends Query
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
+      public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
         assert filter != null;
 
-        final DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
+        DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
         if (filterDocIdSet == null) {
           // this means the filter does not accept any documents.
           return null;
@@ -135,6 +135,20 @@ public class FilteredQuery extends Query
 
         return strategy.filteredScorer(context, weight, filterDocIdSet);
       }
+
+      // return a filtering top scorer
+      @Override
+      public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+        assert filter != null;
+
+        DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
+        if (filterDocIdSet == null) {
+          // this means the filter does not accept any documents.
+          return null;
+        }
+
+        return strategy.filteredTopScorer(context, weight, scoreDocsInOrder, filterDocIdSet);
+      }
     };
   }
   
@@ -147,40 +161,20 @@ public class FilteredQuery extends Query
   private static final class QueryFirstScorer extends Scorer {
     private final Scorer scorer;
     private int scorerDoc = -1;
-    private Bits filterbits;
+    private final Bits filterBits;
 
     protected QueryFirstScorer(Weight weight, Bits filterBits, Scorer other) {
       super(weight);
       this.scorer = other;
-      this.filterbits = filterBits;
+      this.filterBits = filterBits;
     }
 
-    // nocommit move to custom TopScorer
-    // optimization: we are topScorer and collect directly
-    /*
-    @Override
-    public void score(Collector collector) throws IOException {
-      // the normalization trick already applies the boost of this query,
-      // so we can use the wrapped scorer directly:
-      collector.setScorer(scorer);
-      for (;;) {
-        final int scorerDoc = scorer.nextDoc();
-        if (scorerDoc == DocIdSetIterator.NO_MORE_DOCS) {
-          break;
-        }
-        if (filterbits.get(scorerDoc)) {
-          collector.collect(scorerDoc);
-        }
-      }
-    }
-    */
-    
     @Override
     public int nextDoc() throws IOException {
       int doc;
       for(;;) {
         doc = scorer.nextDoc();
-        if (doc == Scorer.NO_MORE_DOCS || filterbits.get(doc)) {
+        if (doc == Scorer.NO_MORE_DOCS || filterBits.get(doc)) {
           return scorerDoc = doc;
         }
       } 
@@ -189,7 +183,7 @@ public class FilteredQuery extends Query
     @Override
     public int advance(int target) throws IOException {
       int doc = scorer.advance(target);
-      if (doc != Scorer.NO_MORE_DOCS && !filterbits.get(doc)) {
+      if (doc != Scorer.NO_MORE_DOCS && !filterBits.get(doc)) {
         return scorerDoc = nextDoc();
       } else {
         return scorerDoc = doc;
@@ -219,6 +213,40 @@ public class FilteredQuery extends Query
       return scorer.cost();
     }
   }
+
+  private static class QueryFirstTopScorer extends TopScorer {
+
+    private final Scorer scorer;
+    private final Bits filterBits;
+
+    public QueryFirstTopScorer(Scorer scorer, Bits filterBits) {
+      this.scorer = scorer;
+      this.filterBits = filterBits;
+    }
+
+    @Override
+    public boolean score(Collector collector, int maxDoc) throws IOException {
+      // the normalization trick already applies the boost of this query,
+      // so we can use the wrapped scorer directly:
+      collector.setScorer(scorer);
+      if (scorer.docID() == -1) {
+        scorer.nextDoc();
+      }
+      while (true) {
+        final int scorerDoc = scorer.docID();
+        if (scorerDoc < maxDoc) {
+          if (filterBits.get(scorerDoc)) {
+            collector.collect(scorerDoc);
+          }
+          scorer.nextDoc();
+        } else {
+          break;
+        }
+      }
+
+      return scorer.docID() != Scorer.NO_MORE_DOCS;
+    }
+  }
   
   /**
    * A Scorer that uses a "leap-frog" approach (also called "zig-zag join"). The scorer and the filter
@@ -240,34 +268,6 @@ public class FilteredQuery extends Query
       this.scorer = scorer;
     }
 
-    // nocommit move to TopScorer
-    // optimization: we are topScorer and collect directly using short-circuited algo
-    /*
-    @Override
-    public final void score(Collector collector) throws IOException {
-      // the normalization trick already applies the boost of this query,
-      // so we can use the wrapped scorer directly:
-      collector.setScorer(scorer);
-      int primDoc = primaryNext();
-      int secDoc = secondary.advance(primDoc);
-      for (;;) {
-        if (primDoc == secDoc) {
-          // Check if scorer has exhausted, only before collecting.
-          if (primDoc == DocIdSetIterator.NO_MORE_DOCS) {
-            break;
-          }
-          collector.collect(primDoc);
-          primDoc = primary.nextDoc();
-          secDoc = secondary.advance(primDoc);
-        } else if (secDoc > primDoc) {
-          primDoc = primary.advance(secDoc);
-        } else {
-          secDoc = secondary.advance(primDoc);
-        }
-      }
-    }
-    */
-    
     private final int advanceToNextCommonDoc() throws IOException {
       for (;;) {
         if (secondaryDoc < primaryDoc) {
@@ -323,6 +323,48 @@ public class FilteredQuery extends Query
       return Math.min(primary.cost(), secondary.cost());
     }
   }
+
+  private static final class LeapFrogTopScorer extends TopScorer {
+    private final DocIdSetIterator primary;
+    private final DocIdSetIterator secondary;
+    private final Scorer scorer;
+
+    public LeapFrogTopScorer(DocIdSetIterator primary, DocIdSetIterator secondary, Scorer scorer) {
+      this.primary = primary;
+      this.secondary = secondary;
+      this.scorer = scorer;
+    }
+
+    @Override
+    public boolean score(Collector collector, int maxDoc) throws IOException {
+      // the normalization trick already applies the boost of this query,
+      // so we can use the wrapped scorer directly:
+      collector.setScorer(scorer);
+      int primDoc, secDoc;
+      if (primary.docID() == -1) {
+        primDoc = primary.nextDoc();
+        secDoc = secondary.advance(primDoc);
+      } else {
+        primDoc = primary.docID();
+        secDoc = secondary.docID();
+      }
+      for (;;) {
+        if (primDoc == secDoc) {
+          // Check if scorer has exhausted, only before collecting.
+          if (primDoc >= maxDoc) {
+            return primDoc < Scorer.NO_MORE_DOCS;
+          }
+          collector.collect(primDoc);
+          primDoc = primary.nextDoc();
+          secDoc = secondary.advance(primDoc);
+        } else if (secDoc > primDoc) {
+          primDoc = primary.advance(secDoc);
+        } else {
+          secDoc = secondary.advance(primDoc);
+        }
+      }
+    }
+  }
   
   // TODO once we have way to figure out if we use RA or LeapFrog we can remove this scorer
   private static final class PrimaryAdvancedLeapFrogScorer extends LeapFrogScorer {
@@ -478,18 +520,6 @@ public class FilteredQuery extends Query
      * 
      * @param context
      *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
-     * @param scoreDocsInOrder
-     *          specifies whether in-order scoring of documents is required. Note
-     *          that if set to false (i.e., out-of-order scoring is required),
-     *          this method can return whatever scoring mode it supports, as every
-     *          in-order scorer is also an out-of-order one. However, an
-     *          out-of-order scorer may not support {@link Scorer#nextDoc()}
-     *          and/or {@link Scorer#advance(int)}, therefore it is recommended to
-     *          request an in-order scorer if use of these methods is required.
-     * @param topScorer
-     *          if true, {@link Scorer#score(Collector)} will be called; if false,
-     *          {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
-     *          be called.
      * @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
      * @param docIdSet the filter {@link DocIdSet} to apply
      * @return a filtered scorer
@@ -498,6 +528,29 @@ public class FilteredQuery extends Query
      */
     public abstract Scorer filteredScorer(AtomicReaderContext context,
         Weight weight, DocIdSet docIdSet) throws IOException;
+
+    /**
+     * Returns a filtered {@link TopScorer} based on this
+     * strategy.  This is an optional method: the default
+     * implementation just calls {@link #filteredScorer} and
+     * wraps that into a TopScorer.
+     *
+     * @param context
+     *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
+     * @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
+     * @param docIdSet the filter {@link DocIdSet} to apply
+     * @return a filtered top scorer
+     */
+    public TopScorer filteredTopScorer(AtomicReaderContext context,
+        Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
+      Scorer scorer = filteredScorer(context, weight, docIdSet);
+      if (scorer == null) {
+        return null;
+      }
+      // This impl always scores docs in order, so we can
+      // ignore scoreDocsInOrder:
+      return new Weight.DefaultTopScorer(scorer);
+    }
   }
   
   /**
@@ -575,13 +628,36 @@ public class FilteredQuery extends Query
       }
       // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
       final Scorer scorer = weight.scorer(context, null);
+      if (scorer == null) {
+        return null;
+      }
+
       if (scorerFirst) {
-        return (scorer == null) ? null : new LeapFrogScorer(weight, scorer, filterIter, scorer);  
+        return new LeapFrogScorer(weight, scorer, filterIter, scorer);  
       } else {
-        return (scorer == null) ? null : new LeapFrogScorer(weight, filterIter, scorer, scorer);  
+        return new LeapFrogScorer(weight, filterIter, scorer, scorer);  
+      }
+    }
+
+    @Override
+    public TopScorer filteredTopScorer(AtomicReaderContext context,
+        Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
+      final DocIdSetIterator filterIter = docIdSet.iterator();
+      if (filterIter == null) {
+        // this means the filter does not accept any documents.
+        return null;
+      }
+      // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
+      final Scorer scorer = weight.scorer(context, null);
+      if (scorer == null) {
+        return null;
+      }
+      if (scorerFirst) {
+        return new LeapFrogTopScorer(scorer, filterIter, scorer);  
+      } else {
+        return new LeapFrogTopScorer(filterIter, scorer, scorer);  
       }
     }
-    
   }
   
   /**
@@ -604,12 +680,29 @@ public class FilteredQuery extends Query
         DocIdSet docIdSet) throws IOException {
       Bits filterAcceptDocs = docIdSet.bits();
       if (filterAcceptDocs == null) {
+        // Filter does not provide random-access Bits; we
+        // must fallback to leapfrog:
         return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet);
       }
       final Scorer scorer = weight.scorer(context, null);
       return scorer == null ? null : new QueryFirstScorer(weight,
           filterAcceptDocs, scorer);
     }
+
+    @Override
+    public TopScorer filteredTopScorer(final AtomicReaderContext context,
+        Weight weight,
+        boolean scoreDocsInOrder, // ignored (we always top-score in order)
+        DocIdSet docIdSet) throws IOException {
+      Bits filterAcceptDocs = docIdSet.bits();
+      if (filterAcceptDocs == null) {
+        // Filter does not provide random-access Bits; we
+        // must fallback to leapfrog:
+        return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredTopScorer(context, weight, scoreDocsInOrder, docIdSet);
+      }
+      final Scorer scorer = weight.scorer(context, null);
+      return scorer == null ? null : new QueryFirstTopScorer(scorer, filterAcceptDocs);
+    }
   }
   
 }

Modified: lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/TopScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/TopScorer.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/TopScorer.java (original)
+++ lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/TopScorer.java Thu Mar  6 21:48:23 2014
@@ -36,7 +36,7 @@ public abstract class TopScorer {
    * @param collector
    *          The collector to which all matching documents are passed.
    * @param max
-   *          Do not score documents past this.
+   *          Score up to, but not including, this doc
    * @return true if more matching documents may remain.
    */
   public abstract boolean score(Collector collector, int max) throws IOException;

Modified: lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/Weight.java Thu Mar  6 21:48:23 2014
@@ -130,7 +130,7 @@ public abstract class Weight {
    */
   public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
 
-    final Scorer scorer = scorer(context, acceptDocs);
+    Scorer scorer = scorer(context, acceptDocs);
     if (scorer == null) {
       // No docs match
       return null;
@@ -138,23 +138,36 @@ public abstract class Weight {
 
     // This impl always scores docs in order, so we can
     // ignore scoreDocsInOrder:
-    return new TopScorer() {
+    return new DefaultTopScorer(scorer);
+  }
+
+  /** Just wraps a Scorer and performs top scoring using it. */
+  static class DefaultTopScorer extends TopScorer {
+    private final Scorer scorer;
+
+    public DefaultTopScorer(Scorer scorer) {
+      assert scorer != null;
+      this.scorer = scorer;
+    }
 
-      @Override
-      public boolean score(Collector collector, int max) throws IOException {
-        // nocommit weird to do this here?  we do it many,
-        // many times from BS1 inside one segment?
-        collector.setScorer(scorer);
-        if (scorer.docID() == -1) {
-          scorer.nextDoc();
-        }
-        int doc;
-        for (doc = scorer.docID(); doc < max; doc = scorer.nextDoc()) {
-          collector.collect(doc);
-        }
-        return doc != DocsEnum.NO_MORE_DOCS;
+    @Override
+    public boolean score(Collector collector, int max) throws IOException {
+      // TODO: this may be sort of weird, when we are
+      // embedded in a BooleanScorer, because we are
+      // called for every chunk of 2048 documents.  But,
+      // then, scorer is a FakeScorer in that case, so any
+      // Collector doing something "interesting" in
+      // setScorer will be forced to use BS2 anyways:
+      collector.setScorer(scorer);
+      if (scorer.docID() == -1) {
+        scorer.nextDoc();
+      }
+      int doc;
+      for (doc = scorer.docID(); doc < max; doc = scorer.nextDoc()) {
+        collector.collect(doc);
       }
-    };
+      return doc != DocsEnum.NO_MORE_DOCS;
+    }
   }
 
   /**

Modified: lucene/dev/branches/lucene5487/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java (original)
+++ lucene/dev/branches/lucene5487/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java Thu Mar  6 21:48:23 2014
@@ -385,10 +385,10 @@ public class TestFilteredQuery extends L
   
   private static FilteredQuery.FilterStrategy randomFilterStrategy(Random random, final boolean useRandomAccess) {
     if (useRandomAccess) {
-      return  new FilteredQuery.RandomAccessFilterStrategy() {
+      return new FilteredQuery.RandomAccessFilterStrategy() {
         @Override
         protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
-          return useRandomAccess;
+          return true;
         }
       };
     }

Modified: lucene/dev/branches/lucene5487/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (original)
+++ lucene/dev/branches/lucene5487/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java Thu Mar  6 21:48:23 2014
@@ -258,8 +258,6 @@ class TermsIncludingScoreQuery extends Q
     int scoreUpto;
     int doc;
 
-    // nocommit can we embed this into a BQ and get BS1?
-
     SVInnerScorer(Weight weight, Bits acceptDocs, TermsEnum termsEnum, long cost) {
       this.acceptDocs = acceptDocs;
       this.termsEnum = termsEnum;

Modified: lucene/dev/branches/lucene5487/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java (original)
+++ lucene/dev/branches/lucene5487/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java Thu Mar  6 21:48:23 2014
@@ -90,8 +90,7 @@ public class AssertingIndexSearcher exte
 
   @Override
   protected void search(List<AtomicReaderContext> leaves, Weight weight, Collector collector) throws IOException {
-    // nocommit shouldn't we
-    // AssertingCollector.wrap(collector) here?
+    // TODO: shouldn't we AssertingCollector.wrap(collector) here?
     super.search(leaves, AssertingWeight.wrap(random, weight), collector);
   }
 

Modified: lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/schema/LatLonType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/schema/LatLonType.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/schema/LatLonType.java (original)
+++ lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/schema/LatLonType.java Thu Mar  6 21:48:23 2014
@@ -333,14 +333,13 @@ class SpatialDistanceQuery extends Exten
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       return new SpatialScorer(context, acceptDocs, this, queryWeight);
     }
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      return ((SpatialScorer)scorer(context, true, true, context.reader().getLiveDocs())).explain(doc);
+      return ((SpatialScorer)scorer(context, context.reader().getLiveDocs())).explain(doc);
     }
   }
 

Modified: lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (original)
+++ lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java Thu Mar  6 21:48:23 2014
@@ -232,8 +232,7 @@ class JoinQuery extends Query {
 
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       if (filter == null) {
         boolean debug = rb != null && rb.isDebug();
         long start = debug ? System.currentTimeMillis() : 0;
@@ -502,7 +501,7 @@ class JoinQuery extends Query {
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, true, false, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, context.reader().getLiveDocs());
       boolean exists = scorer.advance(doc) == doc;
 
       ComplexExplanation result = new ComplexExplanation();

Modified: lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (original)
+++ lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java Thu Mar  6 21:48:23 2014
@@ -119,8 +119,7 @@ public class SolrConstantScoreQuery exte
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       return new ConstantScorer(context, this, queryWeight, acceptDocs);
     }
 

Modified: lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Thu Mar  6 21:48:23 2014
@@ -2472,7 +2472,7 @@ class FilterImpl extends Filter {
         iterators.add(iter);
       }
       for (Weight w : weights) {
-        Scorer scorer = w.scorer(context, true, false, context.reader().getLiveDocs());
+        Scorer scorer = w.scorer(context, context.reader().getLiveDocs());
         if (scorer == null) return null;
         iterators.add(scorer);
       }

Modified: lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java?rev=1575057&r1=1575056&r2=1575057&view=diff
==============================================================================
--- lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java (original)
+++ lucene/dev/branches/lucene5487/solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java Thu Mar  6 21:48:23 2014
@@ -86,8 +86,8 @@ public class IgnoreAcceptDocsQuery exten
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, Bits acceptDocs) throws IOException {
-      return w.scorer(context, scoreDocsInOrder, topScorer, null);
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
+      return w.scorer(context, null);
     }
   }