浏览代码

Core: cutover to Lucene's query rescorer

This is functionally equivalent to before, so there should be no
user-visible impact, except I added a NOTE in the docs warning about
the interaction of pagination and rescoring.

Closes #6232

Closes #7707
Michael McCandless 11 年之前
父节点
当前提交
85065f9c8e

+ 6 - 1
docs/reference/search/request/rescore.asciidoc

@@ -15,10 +15,15 @@ Currently the rescore API has only one implementation: the query
 rescorer, which uses a query to tweak the scoring. In the future, 
 rescorer, which uses a query to tweak the scoring. In the future, 
 alternative rescorers may be made available, for example, a pair-wise rescorer.
 alternative rescorers may be made available, for example, a pair-wise rescorer.
 
 
-*Note:* the `rescore` phase is not executed when
+NOTE: the `rescore` phase is not executed when
 <<search-request-search-type,`search_type`>> is set
 <<search-request-search-type,`search_type`>> is set
 to `scan` or `count`.
 to `scan` or `count`.
 
 
+NOTE: when exposing pagination to your users, you should not change
+`window_size` as you step through each page (by passing different
+`from` values) since that can alter the top hits causing results to
+confusingly shift as the user steps through pages.
+
 ==== Query rescorer
 ==== Query rescorer
 
 
 The query rescorer executes a second query only on the Top-K results
 The query rescorer executes a second query only on the Top-K results

+ 76 - 180
src/main/java/org/elasticsearch/search/rescore/QueryRescorer.java

@@ -19,11 +19,8 @@
 
 
 package org.elasticsearch.search.rescore;
 package org.elasticsearch.search.rescore;
 
 
-import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.*;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.IntroSorter;
 import org.elasticsearch.ElasticsearchIllegalArgumentException;
 import org.elasticsearch.ElasticsearchIllegalArgumentException;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser.Token;
 import org.elasticsearch.common.xcontent.XContentParser.Token;
@@ -33,6 +30,7 @@ import org.elasticsearch.search.internal.SearchContext;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.Set;
 import java.util.Set;
 
 
 public final class QueryRescorer implements Rescorer {
 public final class QueryRescorer implements Rescorer {
@@ -106,28 +104,49 @@ public final class QueryRescorer implements Rescorer {
     }
     }
 
 
     @Override
     @Override
-    public void rescore(TopDocs topDocs, SearchContext context, RescoreSearchContext rescoreContext) throws IOException {
+    public TopDocs rescore(TopDocs topDocs, SearchContext context, RescoreSearchContext rescoreContext) throws IOException {
+
         assert rescoreContext != null;
         assert rescoreContext != null;
         if (topDocs == null || topDocs.totalHits == 0 || topDocs.scoreDocs.length == 0) {
         if (topDocs == null || topDocs.totalHits == 0 || topDocs.scoreDocs.length == 0) {
-            return;
+            return topDocs;
         }
         }
 
 
-        QueryRescoreContext rescore = (QueryRescoreContext) rescoreContext;
-        ContextIndexSearcher searcher = context.searcher();
-        TopDocsFilter filter = new TopDocsFilter(topDocs, rescoreContext.window());
-        TopDocs rescored = searcher.search(rescore.query(), filter, rescoreContext.window());
-        context.queryResult().topDocs(merge(topDocs, rescored, rescore));
+        final QueryRescoreContext rescore = (QueryRescoreContext) rescoreContext;
+
+        org.apache.lucene.search.Rescorer rescorer = new org.apache.lucene.search.QueryRescorer(rescore.query()) {
+
+            @Override
+            protected float combine(float firstPassScore, boolean secondPassMatches, float secondPassScore) {
+                if (secondPassMatches) {
+                    return rescore.scoreMode.combine(firstPassScore * rescore.queryWeight(), secondPassScore * rescore.rescoreQueryWeight());
+                }
+                // TODO: shouldn't this be up to the ScoreMode?  I.e., we should just invoke ScoreMode.combine, passing 0.0f for the
+                // secondary score?
+                return firstPassScore * rescore.queryWeight();
+            }
+        };
+
+        // First take top slice of incoming docs, to be rescored:
+        TopDocs topNFirstPass = topN(topDocs, rescoreContext.window());
+
+        // Rescore them:
+        TopDocs rescored = rescorer.rescore(context.searcher(), topNFirstPass, rescoreContext.window());
+
+        // Splice back to non-topN hits and resort all of them:
+        return combine(topDocs, rescored, (QueryRescoreContext) rescoreContext);
     }
     }
 
 
     @Override
     @Override
     public Explanation explain(int topLevelDocId, SearchContext context, RescoreSearchContext rescoreContext,
     public Explanation explain(int topLevelDocId, SearchContext context, RescoreSearchContext rescoreContext,
-            Explanation sourceExplanation) throws IOException {
-        QueryRescoreContext rescore = ((QueryRescoreContext) rescoreContext);
+                               Explanation sourceExplanation) throws IOException {
+        QueryRescoreContext rescore = (QueryRescoreContext) rescoreContext;
         ContextIndexSearcher searcher = context.searcher();
         ContextIndexSearcher searcher = context.searcher();
         if (sourceExplanation == null) {
         if (sourceExplanation == null) {
             // this should not happen but just in case
             // this should not happen but just in case
             return new ComplexExplanation(false, 0.0f, "nothing matched");
             return new ComplexExplanation(false, 0.0f, "nothing matched");
         }
         }
+        // TODO: this isn't right?  I.e., we are incorrectly pretending all first pass hits were rescored?  If the requested docID was
+        // beyond the top rescoreContext.window() in the first pass hits, we don't rescore it now?
         Explanation rescoreExplain = searcher.explain(rescore.query(), topLevelDocId);
         Explanation rescoreExplain = searcher.explain(rescore.query(), topLevelDocId);
         float primaryWeight = rescore.queryWeight();
         float primaryWeight = rescore.queryWeight();
         ComplexExplanation prim = new ComplexExplanation(sourceExplanation.isMatch(),
         ComplexExplanation prim = new ComplexExplanation(sourceExplanation.isMatch(),
@@ -135,6 +154,9 @@ public final class QueryRescorer implements Rescorer {
                 "product of:");
                 "product of:");
         prim.addDetail(sourceExplanation);
         prim.addDetail(sourceExplanation);
         prim.addDetail(new Explanation(primaryWeight, "primaryWeight"));
         prim.addDetail(new Explanation(primaryWeight, "primaryWeight"));
+
+        // NOTE: we don't use Lucene's Rescorer.explain because we want to insert our own description with which ScoreMode was used.  Maybe
+        // we should add QueryRescorer.explainCombine to Lucene?
         if (rescoreExplain != null && rescoreExplain.isMatch()) {
         if (rescoreExplain != null && rescoreExplain.isMatch()) {
             float secondaryWeight = rescore.rescoreQueryWeight();
             float secondaryWeight = rescore.rescoreQueryWeight();
             ComplexExplanation sec = new ComplexExplanation(rescoreExplain.isMatch(),
             ComplexExplanation sec = new ComplexExplanation(rescoreExplain.isMatch(),
@@ -195,6 +217,48 @@ public final class QueryRescorer implements Rescorer {
         return rescoreContext;
         return rescoreContext;
     }
     }
 
 
+    private final static Comparator<ScoreDoc> SCORE_DOC_COMPARATOR = new Comparator<ScoreDoc>() {
+        @Override
+        public int compare(ScoreDoc o1, ScoreDoc o2) {
+            int cmp = Float.compare(o2.score, o1.score);
+            return cmp == 0 ?  Integer.compare(o1.doc, o2.doc) : cmp;
+        }
+    };
+
+    /** Returns a new {@link TopDocs} with the topN from the incoming one, or the same TopDocs if the number of hits is already <=
+     *  topN. */
+    private TopDocs topN(TopDocs in, int topN) {
+        if (in.totalHits < topN) {
+            assert in.scoreDocs.length == in.totalHits;
+            return in;
+        }
+
+        ScoreDoc[] subset = new ScoreDoc[topN];
+        System.arraycopy(in.scoreDocs, 0, subset, 0, topN);
+
+        return new TopDocs(in.totalHits, subset, in.getMaxScore());
+    }
+
+    /** Modifies incoming TopDocs (in) by replacing the top hits with resorted's hits, and then resorting all hits. */
+    private TopDocs combine(TopDocs in, TopDocs resorted, QueryRescoreContext ctx) {
+
+        System.arraycopy(resorted.scoreDocs, 0, in.scoreDocs, 0, resorted.scoreDocs.length);
+        if (in.scoreDocs.length > resorted.scoreDocs.length) {
+            // These hits were not rescored (beyond the rescore window), so we treat them the same as a hit that did get rescored but did
+            // not match the 2nd pass query:
+            for(int i=resorted.scoreDocs.length;i<in.scoreDocs.length;i++) {
+                // TODO: shouldn't this be up to the ScoreMode?  I.e., we should just invoke ScoreMode.combine, passing 0.0f for the
+                // secondary score?
+                in.scoreDocs[i].score *= ctx.queryWeight();
+            }
+            
+            // TODO: this is wrong, i.e. we are comparing apples and oranges at this point.  It would be better if we always rescored all
+            // incoming first pass hits, instead of allowing recoring of just the top subset:
+            Arrays.sort(in.scoreDocs, SCORE_DOC_COMPARATOR);
+        }
+        return in;
+    }
+
     public static class QueryRescoreContext extends RescoreSearchContext {
     public static class QueryRescoreContext extends RescoreSearchContext {
 
 
         public QueryRescoreContext(QueryRescorer rescorer) {
         public QueryRescoreContext(QueryRescorer rescorer) {
@@ -241,174 +305,6 @@ public final class QueryRescorer implements Rescorer {
 
 
     }
     }
 
 
-
-    private TopDocs merge(TopDocs primary, TopDocs secondary, QueryRescoreContext context) {
-        DocIdSorter sorter = new DocIdSorter();
-        sorter.array = primary.scoreDocs;
-        sorter.sort(0, sorter.array.length);
-        ScoreDoc[] primaryDocs = sorter.array;
-        sorter.array = secondary.scoreDocs;
-        sorter.sort(0, sorter.array.length);
-        ScoreDoc[] secondaryDocs = sorter.array;
-        int j = 0;
-        float primaryWeight = context.queryWeight();
-        float secondaryWeight = context.rescoreQueryWeight();
-        ScoreMode scoreMode = context.scoreMode();
-        for (int i = 0; i < primaryDocs.length; i++) {
-            if (j < secondaryDocs.length && primaryDocs[i].doc == secondaryDocs[j].doc) {
-                primaryDocs[i].score = scoreMode.combine(primaryDocs[i].score * primaryWeight, secondaryDocs[j++].score * secondaryWeight);
-            } else {
-                primaryDocs[i].score *= primaryWeight;
-            }
-        }
-        ScoreSorter scoreSorter = new ScoreSorter();
-        scoreSorter.array = primaryDocs;
-        scoreSorter.sort(0, primaryDocs.length);
-        primary.setMaxScore(primaryDocs[0].score);
-        return primary;
-    }
-
-    private static final class DocIdSorter extends IntroSorter {
-        private ScoreDoc[] array;
-        private ScoreDoc pivot;
-
-        @Override
-        protected void swap(int i, int j) {
-            ScoreDoc scoreDoc = array[i];
-            array[i] = array[j];
-            array[j] = scoreDoc;
-        }
-
-        @Override
-        protected int compare(int i, int j) {
-            return compareDocId(array[i], array[j]);
-        }
-
-        @Override
-        protected void setPivot(int i) {
-            pivot = array[i];
-
-        }
-
-        @Override
-        protected int comparePivot(int j) {
-            return compareDocId(pivot, array[j]);
-        }
-
-    }
-
-    private static final int compareDocId(ScoreDoc left, ScoreDoc right) {
-        if (left.doc < right.doc) {
-            return 1;
-        } else if (left.doc == right.doc) {
-            return 0;
-        }
-        return -1;
-    }
-
-    private static final class ScoreSorter extends IntroSorter {
-        private ScoreDoc[] array;
-        private ScoreDoc pivot;
-
-        @Override
-        protected void swap(int i, int j) {
-            ScoreDoc scoreDoc = array[i];
-            array[i] = array[j];
-            array[j] = scoreDoc;
-        }
-
-        @Override
-        protected int compare(int i, int j) {
-            int cmp = Float.compare(array[j].score, array[i].score);
-            return cmp == 0 ? compareDocId(array[i], array[j]) : cmp;
-        }
-
-        @Override
-        protected void setPivot(int i) {
-            pivot = array[i];
-
-        }
-
-        @Override
-        protected int comparePivot(int j) {
-            int cmp = Float.compare(array[j].score, pivot.score);
-            return cmp == 0 ? compareDocId(pivot, array[j]) : cmp;
-        }
-
-    }
-
-    private static final class TopDocsFilter extends Filter {
-
-        private final int[] docIds;
-
-        public TopDocsFilter(TopDocs topDocs, int max) {
-            ScoreDoc[] scoreDocs = topDocs.scoreDocs;
-            max = Math.min(max, scoreDocs.length);
-            this.docIds = new int[max];
-            for (int i = 0; i < max; i++) {
-                docIds[i] = scoreDocs[i].doc;
-            }
-            Arrays.sort(docIds);
-        }
-
-        @Override
-        public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-            final int docBase = context.docBase;
-            int limit = docBase + context.reader().maxDoc();
-            int offset = Arrays.binarySearch(docIds, docBase);
-            if (offset < 0) {
-                offset = (-offset) - 1;
-            }
-            int end = Arrays.binarySearch(docIds, limit);
-            if (end < 0) {
-                end = (-end) - 1;
-            }
-            final int start = offset;
-            final int stop = end;
-
-            return new DocIdSet() {
-
-                @Override
-                public DocIdSetIterator iterator() throws IOException {
-                    return new DocIdSetIterator() {
-                        private int current = start;
-                        private int docId = NO_MORE_DOCS;
-
-                        @Override
-                        public int nextDoc() throws IOException {
-                            if (current < stop) {
-                                return docId = docIds[current++] - docBase;
-                            }
-                            return docId = NO_MORE_DOCS;
-                        }
-
-                        @Override
-                        public int docID() {
-                            return docId;
-                        }
-
-                        @Override
-                        public int advance(int target) throws IOException {
-                            if (target == NO_MORE_DOCS) {
-                                current = stop;
-                                return docId = NO_MORE_DOCS;
-                            }
-                            while (nextDoc() < target) {
-                            }
-                            return docId;
-                        }
-
-                        @Override
-                        public long cost() {
-                            return docIds.length;
-                        }
-                    };
-                }
-            };
-        }
-
-    }
-
     @Override
     @Override
     public void extractTerms(SearchContext context, RescoreSearchContext rescoreContext, Set<Term> termsSet) {
     public void extractTerms(SearchContext context, RescoreSearchContext rescoreContext, Set<Term> termsSet) {
         ((QueryRescoreContext) rescoreContext).query().extractTerms(termsSet);
         ((QueryRescoreContext) rescoreContext).query().extractTerms(termsSet);

+ 10 - 6
src/main/java/org/elasticsearch/search/rescore/RescorePhase.java

@@ -20,6 +20,8 @@
 package org.elasticsearch.search.rescore;
 package org.elasticsearch.search.rescore;
 
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMap;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.inject.Inject;
@@ -54,16 +56,18 @@ public class RescorePhase extends AbstractComponent implements SearchPhase {
     @Override
     @Override
     public void execute(SearchContext context) throws ElasticsearchException {
     public void execute(SearchContext context) throws ElasticsearchException {
         try {
         try {
+            TopDocs topDocs = context.queryResult().topDocs();
             for (RescoreSearchContext ctx : context.rescore()) {
             for (RescoreSearchContext ctx : context.rescore()) {
-                ctx.rescorer().rescore(context.queryResult().topDocs(), context, ctx);
+                topDocs = ctx.rescorer().rescore(topDocs, context, ctx);
             }
             }
+            if (context.size() < topDocs.scoreDocs.length) {
+                ScoreDoc[] hits = new ScoreDoc[context.size()];
+                System.arraycopy(topDocs.scoreDocs, 0, hits, 0, hits.length);
+                topDocs = new TopDocs(topDocs.totalHits, hits, topDocs.getMaxScore());
+            }
+            context.queryResult().topDocs(topDocs);
         } catch (IOException e) {
         } catch (IOException e) {
             throw new ElasticsearchException("Rescore Phase Failed", e);
             throw new ElasticsearchException("Rescore Phase Failed", e);
         }
         }
     }
     }
-  
-    
-
-    
-
 }
 }

+ 1 - 1
src/main/java/org/elasticsearch/search/rescore/Rescorer.java

@@ -49,7 +49,7 @@ public interface Rescorer {
      * @param rescoreContext the {@link RescoreSearchContext}. This will never be <code>null</code>
      * @param rescoreContext the {@link RescoreSearchContext}. This will never be <code>null</code>
      * @throws IOException if an {@link IOException} occurs during rescoring
      * @throws IOException if an {@link IOException} occurs during rescoring
      */
      */
-    public void rescore(TopDocs topDocs, SearchContext context, RescoreSearchContext rescoreContext) throws IOException;
+    public TopDocs rescore(TopDocs topDocs, SearchContext context, RescoreSearchContext rescoreContext) throws IOException;
 
 
     /**
     /**
      * Executes an {@link Explanation} phase on the rescorer.
      * Executes an {@link Explanation} phase on the rescorer.

+ 164 - 2
src/test/java/org/elasticsearch/search/rescore/QueryRescorerTests.java

@@ -28,8 +28,8 @@ import org.elasticsearch.action.search.SearchRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.common.lucene.search.function.CombineFunction;
 import org.elasticsearch.common.lucene.search.function.CombineFunction;
-import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.ImmutableSettings.Builder;
 import org.elasticsearch.common.settings.ImmutableSettings.Builder;
+import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.index.query.MatchQueryBuilder;
 import org.elasticsearch.index.query.MatchQueryBuilder;
@@ -38,6 +38,7 @@ import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchHits;
 import org.elasticsearch.search.SearchHits;
 import org.elasticsearch.search.rescore.RescoreBuilder.QueryRescorer;
 import org.elasticsearch.search.rescore.RescoreBuilder.QueryRescorer;
+import org.elasticsearch.search.sort.SortOrder;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -83,7 +84,6 @@ public class QueryRescorerTests extends ElasticsearchIntegrationTest {
                 }
                 }
             }
             }
         }
         }
-
     }
     }
 
 
     @Test
     @Test
@@ -193,6 +193,168 @@ public class QueryRescorerTests extends ElasticsearchIntegrationTest {
         assertFirstHit(searchResponse, hasId("2"));
         assertFirstHit(searchResponse, hasId("2"));
         assertSecondHit(searchResponse, hasId("6"));
         assertSecondHit(searchResponse, hasId("6"));
         assertThirdHit(searchResponse, hasId("3"));
         assertThirdHit(searchResponse, hasId("3"));
+
+        // Make sure non-zero from works:
+        searchResponse = client()
+                .prepareSearch()
+                .setQuery(QueryBuilders.matchQuery("field1", "lexington avenue massachusetts").operator(MatchQueryBuilder.Operator.OR))
+                .setFrom(2)
+                .setSize(5)
+                .setSearchType(SearchType.DFS_QUERY_THEN_FETCH)
+                .setRescorer(
+                        RescoreBuilder.queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3))
+                                .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f)).setRescoreWindow(20).execute().actionGet();
+
+        assertThat(searchResponse.getHits().hits().length, equalTo(3));
+        assertHitCount(searchResponse, 9);
+        assertFirstHit(searchResponse, hasId("3"));
+    }
+
+    // Tests a rescore window smaller than number of hits:
+    @Test
+    public void testSmallRescoreWindow() throws Exception {
+        Builder builder = ImmutableSettings.builder();
+        builder.put("index.analysis.analyzer.synonym.tokenizer", "whitespace");
+        builder.putArray("index.analysis.analyzer.synonym.filter", "synonym", "lowercase");
+        builder.put("index.analysis.filter.synonym.type", "synonym");
+        builder.putArray("index.analysis.filter.synonym.synonyms", "ave => ave, avenue", "street => str, street");
+
+        XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
+                .startObject("field1").field("type", "string").field("index_analyzer", "whitespace").field("search_analyzer", "synonym")
+                .endObject().endObject().endObject().endObject();
+
+        assertAcked(client().admin().indices().prepareCreate("test").addMapping("type1", mapping).setSettings(builder.put("index.number_of_shards", 1)));
+
+        client().prepareIndex("test", "type1", "1").setSource("field1", "massachusetts avenue boston massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "2").setSource("field1", "lexington avenue boston massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "3").setSource("field1", "boston avenue lexington massachusetts").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+        client().prepareIndex("test", "type1", "4").setSource("field1", "boston road lexington massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "5").setSource("field1", "lexington street lexington massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "6").setSource("field1", "massachusetts avenue lexington massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "7").setSource("field1", "bosten street san franciso california").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+        client().prepareIndex("test", "type1", "8").setSource("field1", "hollywood boulevard los angeles california").execute().actionGet();
+        client().prepareIndex("test", "type1", "9").setSource("field1", "1st street boston massachussetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "10").setSource("field1", "1st street boston massachusetts").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+        client().prepareIndex("test", "type1", "11").setSource("field1", "2st street boston massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "12").setSource("field1", "3st street boston massachusetts").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+
+        SearchResponse searchResponse = client()
+                .prepareSearch()
+                .setQuery(QueryBuilders.matchQuery("field1", "massachusetts").operator(MatchQueryBuilder.Operator.OR))
+                .setFrom(0)
+            .setSize(5).execute().actionGet();
+        assertThat(searchResponse.getHits().hits().length, equalTo(5));
+        assertHitCount(searchResponse, 9);
+        assertFirstHit(searchResponse, hasId("1"));
+        assertSecondHit(searchResponse, hasId("6"));
+        assertThirdHit(searchResponse, hasId("2"));
+        assertFourthHit(searchResponse, hasId("3"));
+        assertFifthHit(searchResponse, hasId("4"));
+
+        // Now, rescore only top 2 hits w/ proximity:
+        searchResponse = client()
+                .prepareSearch()
+                .setQuery(QueryBuilders.matchQuery("field1", "massachusetts").operator(MatchQueryBuilder.Operator.OR))
+                .setFrom(0)
+                .setSize(5)
+                .setRescorer(
+                        RescoreBuilder.queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3))
+                                .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f)).setRescoreWindow(2).execute().actionGet();
+
+        // Only top 2 hits were re-ordered:
+        assertThat(searchResponse.getHits().hits().length, equalTo(5));
+        assertHitCount(searchResponse, 9);
+        assertFirstHit(searchResponse, hasId("6"));
+        assertSecondHit(searchResponse, hasId("1"));
+        assertThirdHit(searchResponse, hasId("2"));
+        assertFourthHit(searchResponse, hasId("3"));
+        assertFifthHit(searchResponse, hasId("4"));
+
+        // Now, rescore only top 3 hits w/ proximity:
+        searchResponse = client()
+                .prepareSearch()
+                .setQuery(QueryBuilders.matchQuery("field1", "massachusetts").operator(MatchQueryBuilder.Operator.OR))
+                .setFrom(0)
+                .setSize(5)
+                .setRescorer(
+                        RescoreBuilder.queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3))
+                                .setQueryWeight(0.6f).setRescoreQueryWeight(2.0f)).setRescoreWindow(3).execute().actionGet();
+
+        // Only top 3 hits were re-ordered:
+        assertThat(searchResponse.getHits().hits().length, equalTo(5));
+        assertHitCount(searchResponse, 9);
+        assertFirstHit(searchResponse, hasId("2"));
+        assertSecondHit(searchResponse, hasId("6"));
+        assertThirdHit(searchResponse, hasId("1"));
+        assertFourthHit(searchResponse, hasId("3"));
+        assertFifthHit(searchResponse, hasId("4"));
+    }
+
+    // Tests a rescorer that penalizes the scores:
+    @Test
+    public void testRescorerMadeScoresWorse() throws Exception {
+        Builder builder = ImmutableSettings.builder();
+        builder.put("index.analysis.analyzer.synonym.tokenizer", "whitespace");
+        builder.putArray("index.analysis.analyzer.synonym.filter", "synonym", "lowercase");
+        builder.put("index.analysis.filter.synonym.type", "synonym");
+        builder.putArray("index.analysis.filter.synonym.synonyms", "ave => ave, avenue", "street => str, street");
+
+        XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
+                .startObject("field1").field("type", "string").field("index_analyzer", "whitespace").field("search_analyzer", "synonym")
+                .endObject().endObject().endObject().endObject();
+
+        assertAcked(client().admin().indices().prepareCreate("test").addMapping("type1", mapping).setSettings(builder.put("index.number_of_shards", 1)));
+
+        client().prepareIndex("test", "type1", "1").setSource("field1", "massachusetts avenue boston massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "2").setSource("field1", "lexington avenue boston massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "3").setSource("field1", "boston avenue lexington massachusetts").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+        client().prepareIndex("test", "type1", "4").setSource("field1", "boston road lexington massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "5").setSource("field1", "lexington street lexington massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "6").setSource("field1", "massachusetts avenue lexington massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "7").setSource("field1", "bosten street san franciso california").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+        client().prepareIndex("test", "type1", "8").setSource("field1", "hollywood boulevard los angeles california").execute().actionGet();
+        client().prepareIndex("test", "type1", "9").setSource("field1", "1st street boston massachussetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "10").setSource("field1", "1st street boston massachusetts").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+        client().prepareIndex("test", "type1", "11").setSource("field1", "2st street boston massachusetts").execute().actionGet();
+        client().prepareIndex("test", "type1", "12").setSource("field1", "3st street boston massachusetts").execute().actionGet();
+        client().admin().indices().prepareRefresh("test").execute().actionGet();
+
+        SearchResponse searchResponse = client()
+                .prepareSearch()
+                .setQuery(QueryBuilders.matchQuery("field1", "massachusetts").operator(MatchQueryBuilder.Operator.OR))
+                .setFrom(0)
+            .setSize(5).execute().actionGet();
+        assertThat(searchResponse.getHits().hits().length, equalTo(5));
+        assertHitCount(searchResponse, 9);
+        assertFirstHit(searchResponse, hasId("1"));
+        assertSecondHit(searchResponse, hasId("6"));
+        assertThirdHit(searchResponse, hasId("2"));
+        assertFourthHit(searchResponse, hasId("3"));
+        assertFifthHit(searchResponse, hasId("4"));
+
+        // Now, penalizing rescore (nothing matches the rescore query):
+        searchResponse = client()
+                .prepareSearch()
+                .setQuery(QueryBuilders.matchQuery("field1", "massachusetts").operator(MatchQueryBuilder.Operator.OR))
+                .setFrom(0)
+                .setSize(5)
+                .setRescorer(
+                        RescoreBuilder.queryRescorer(QueryBuilders.matchPhraseQuery("field1", "lexington avenue massachusetts").slop(3))
+                                .setQueryWeight(1.0f).setRescoreQueryWeight(-1f)).setRescoreWindow(3).execute().actionGet();
+
+        // 6 and 2 got worse, and then hits outside the rescore window were sorted ahead:
+        assertFirstHit(searchResponse, hasId("1"));
+        assertSecondHit(searchResponse, hasId("3"));
+        assertThirdHit(searchResponse, hasId("4"));
+        assertFourthHit(searchResponse, hasId("6"));
+        assertFifthHit(searchResponse, hasId("2"));
     }
     }
 
 
     // Comparator that sorts hits and rescored hits in the same way.
     // Comparator that sorts hits and rescored hits in the same way.

+ 8 - 0
src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java

@@ -216,6 +216,14 @@ public class ElasticsearchAssertions {
         assertSearchHit(searchResponse, 3, matcher);
         assertSearchHit(searchResponse, 3, matcher);
     }
     }
 
 
+    public static void assertFourthHit(SearchResponse searchResponse, Matcher<SearchHit> matcher) {
+        assertSearchHit(searchResponse, 4, matcher);
+    }
+
+    public static void assertFifthHit(SearchResponse searchResponse, Matcher<SearchHit> matcher) {
+        assertSearchHit(searchResponse, 5, matcher);
+    }
+
     public static void assertSearchHit(SearchResponse searchResponse, int number, Matcher<SearchHit> matcher) {
     public static void assertSearchHit(SearchResponse searchResponse, int number, Matcher<SearchHit> matcher) {
         assertThat(number, greaterThan(0));
         assertThat(number, greaterThan(0));
         assertThat("SearchHit number must be greater than 0", number, greaterThan(0));
         assertThat("SearchHit number must be greater than 0", number, greaterThan(0));