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

[4/4] lucene-solr:branch_7x: LUCENE-8249: Implement Matches API for phrase queries

LUCENE-8249: Implement Matches API for phrase queries


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/67056328
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/67056328
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/67056328

Branch: refs/heads/branch_7x
Commit: 6705632810f700e81ab04a9516ad42feb35aaf64
Parents: 9eea96a
Author: Alan Woodward <ro...@apache.org>
Authored: Thu May 10 09:35:19 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu May 10 12:05:32 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../lucene/search/ExactPhraseMatcher.java       | 152 +++++
 .../apache/lucene/search/ExactPhraseScorer.java | 170 ------
 .../apache/lucene/search/MultiPhraseQuery.java  | 327 ++++++----
 .../org/apache/lucene/search/PhraseMatcher.java |  93 +++
 .../org/apache/lucene/search/PhraseQuery.java   | 203 +++----
 .../org/apache/lucene/search/PhraseScorer.java  |  85 +++
 .../org/apache/lucene/search/PhraseWeight.java  | 124 ++++
 .../lucene/search/SloppyPhraseMatcher.java      | 597 +++++++++++++++++++
 .../lucene/search/SloppyPhraseScorer.java       | 587 ------------------
 .../apache/lucene/search/TestBooleanQuery.java  |   2 +-
 .../apache/lucene/search/TestBooleanScorer.java |   2 +-
 .../lucene/search/TestMatchesIterator.java      |  72 ++-
 .../lucene/search/TestSloppyPhraseQuery.java    |  32 +-
 14 files changed, 1430 insertions(+), 1019 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a118f3e..8dc3076 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -52,6 +52,9 @@ New Features
   This allows exact hit extraction and will enable implementation of accurate 
   highlighters. (Alan Woodward, Adrien Grand, David Smiley)
 
+* LUCENE-8249: Implement Matches API for phrase queries (Alan Woodward, Adrien 
+  Grand)
+
 * LUCENE-8246: Allow to customize the number of deletes a merge claims. This
   helps merge policies in the soft-delete case to correctly implement retention
   policies without triggering uncessary merges. (Simon Willnauer, Mike McCandless)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/ExactPhraseMatcher.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ExactPhraseMatcher.java b/lucene/core/src/java/org/apache/lucene/search/ExactPhraseMatcher.java
new file mode 100644
index 0000000..693ec23
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/ExactPhraseMatcher.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.search.similarities.Similarity;
+
+final class ExactPhraseMatcher extends PhraseMatcher {
+
+  private static class PostingsAndPosition {
+    private final PostingsEnum postings;
+    private final int offset;
+    private int freq, upTo, pos;
+
+    public PostingsAndPosition(PostingsEnum postings, int offset) {
+      this.postings = postings;
+      this.offset = offset;
+    }
+  }
+
+  private final PostingsAndPosition[] postings;
+
+  ExactPhraseMatcher(PhraseQuery.PostingsAndFreq[] postings, float matchCost) {
+    super(approximation(postings), matchCost);
+    List<PostingsAndPosition> postingsAndPositions = new ArrayList<>();
+    for(PhraseQuery.PostingsAndFreq posting : postings) {
+      postingsAndPositions.add(new PostingsAndPosition(posting.postings, posting.position));
+    }
+    this.postings = postingsAndPositions.toArray(new PostingsAndPosition[postingsAndPositions.size()]);
+  }
+
+  private static DocIdSetIterator approximation(PhraseQuery.PostingsAndFreq[] postings) {
+    List<DocIdSetIterator> iterators = new ArrayList<>();
+    for (PhraseQuery.PostingsAndFreq posting : postings) {
+      iterators.add(posting.postings);
+    }
+    return ConjunctionDISI.intersectIterators(iterators);
+  }
+
+  @Override
+  float maxFreq() {
+    int minFreq = postings[0].freq;
+    for (int i = 1; i < postings.length; i++) {
+      minFreq = Math.min(minFreq, postings[i].freq);
+    }
+    return minFreq;
+  }
+
+  /** Advance the given pos enum to the first doc on or after {@code target}.
+   *  Return {@code false} if the enum was exhausted before reaching
+   *  {@code target} and {@code true} otherwise. */
+  private static boolean advancePosition(PostingsAndPosition posting, int target) throws IOException {
+    while (posting.pos < target) {
+      if (posting.upTo == posting.freq) {
+        return false;
+      } else {
+        posting.pos = posting.postings.nextPosition();
+        posting.upTo += 1;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    for (PostingsAndPosition posting : postings) {
+      posting.freq = posting.postings.freq();
+      posting.pos = -1;
+      posting.upTo = 0;
+    }
+  }
+
+  @Override
+  public boolean nextMatch() throws IOException {
+    final PostingsAndPosition lead = postings[0];
+    if (lead.upTo < lead.freq) {
+      lead.pos = lead.postings.nextPosition();
+      lead.upTo += 1;
+    }
+    else {
+      return false;
+    }
+    advanceHead:
+    while (true) {
+      final int phrasePos = lead.pos - lead.offset;
+      for (int j = 1; j < postings.length; ++j) {
+        final PostingsAndPosition posting = postings[j];
+        final int expectedPos = phrasePos + posting.offset;
+
+        // advance up to the same position as the lead
+        if (advancePosition(posting, expectedPos) == false) {
+          break advanceHead;
+        }
+
+        if (posting.pos != expectedPos) { // we advanced too far
+          if (advancePosition(lead, posting.pos - posting.offset + lead.offset)) {
+            continue advanceHead;
+          } else {
+            break advanceHead;
+          }
+        }
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  float sloppyWeight(Similarity.SimScorer simScorer) {
+    return 1;
+  }
+
+  @Override
+  public int startPosition() {
+    return postings[0].pos;
+  }
+
+  @Override
+  public int endPosition() {
+    return postings[postings.length - 1].pos;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return postings[0].postings.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return postings[postings.length - 1].postings.endOffset();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java b/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
deleted file mode 100644
index f54f5f5..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search;
-
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.search.similarities.Similarity;
-
-final class ExactPhraseScorer extends Scorer {
-
-  private static class PostingsAndPosition {
-    private final PostingsEnum postings;
-    private final int offset;
-    private int freq, upTo, pos;
-
-    public PostingsAndPosition(PostingsEnum postings, int offset) {
-      this.postings = postings;
-      this.offset = offset;
-    }
-  }
-
-  private final DocIdSetIterator conjunction;
-  private final PostingsAndPosition[] postings;
-
-  private int freq;
-
-  private final Similarity.SimScorer docScorer;
-  private final boolean needsScores;
-  private float matchCost;
-
-  ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-                    Similarity.SimScorer docScorer, boolean needsScores,
-                    float matchCost) throws IOException {
-    super(weight);
-    this.docScorer = docScorer;
-    this.needsScores = needsScores;
-
-    List<DocIdSetIterator> iterators = new ArrayList<>();
-    List<PostingsAndPosition> postingsAndPositions = new ArrayList<>();
-    for(PhraseQuery.PostingsAndFreq posting : postings) {
-      iterators.add(posting.postings);
-      postingsAndPositions.add(new PostingsAndPosition(posting.postings, posting.position));
-    }
-    conjunction = ConjunctionDISI.intersectIterators(iterators);
-    assert TwoPhaseIterator.unwrap(conjunction) == null;
-    this.postings = postingsAndPositions.toArray(new PostingsAndPosition[postingsAndPositions.size()]);
-    this.matchCost = matchCost;
-  }
-
-  @Override
-  public TwoPhaseIterator twoPhaseIterator() {
-    return new TwoPhaseIterator(conjunction) {
-      @Override
-      public boolean matches() throws IOException {
-        return phraseFreq() > 0;
-      }
-
-      @Override
-      public float matchCost() {
-        return matchCost;
-      }
-    };
-  }
-
-  @Override
-  public DocIdSetIterator iterator() {
-    return TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator());
-  }
-
-  @Override
-  public String toString() {
-    return "ExactPhraseScorer(" + weight + ")";
-  }
-
-  final int freq() {
-    return freq;
-  }
-
-  @Override
-  public int docID() {
-    return conjunction.docID();
-  }
-
-  @Override
-  public float score() throws IOException {
-    return docScorer.score(docID(), freq);
-  }
-
-  /** Advance the given pos enum to the first doc on or after {@code target}.
-   *  Return {@code false} if the enum was exhausted before reaching
-   *  {@code target} and {@code true} otherwise. */
-  private static boolean advancePosition(PostingsAndPosition posting, int target) throws IOException {
-    while (posting.pos < target) {
-      if (posting.upTo == posting.freq) {
-        return false;
-      } else {
-        posting.pos = posting.postings.nextPosition();
-        posting.upTo += 1;
-      }
-    }
-    return true;
-  }
-
-  private int phraseFreq() throws IOException {
-    // reset state
-    final PostingsAndPosition[] postings = this.postings;
-    for (PostingsAndPosition posting : postings) {
-      posting.freq = posting.postings.freq();
-      posting.pos = posting.postings.nextPosition();
-      posting.upTo = 1;
-    }
-
-    int freq = 0;
-    final PostingsAndPosition lead = postings[0];
-
-    advanceHead:
-    while (true) {
-      final int phrasePos = lead.pos - lead.offset;
-      for (int j = 1; j < postings.length; ++j) {
-        final PostingsAndPosition posting = postings[j];
-        final int expectedPos = phrasePos + posting.offset;
-
-        // advance up to the same position as the lead
-        if (advancePosition(posting, expectedPos) == false) {
-          break advanceHead;
-        }
-
-        if (posting.pos != expectedPos) { // we advanced too far
-          if (advancePosition(lead, posting.pos - posting.offset + lead.offset)) {
-            continue advanceHead;
-          } else {
-            break advanceHead;
-          }
-        }
-      }
-
-      freq += 1;
-      if (needsScores == false) {
-        break;
-      }
-
-      if (lead.upTo == lead.freq) {
-        break;
-      }
-      lead.pos = lead.postings.nextPosition();
-      lead.upTo += 1;
-    }
-
-    return this.freq = freq;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
index f983c9d..1374f45 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
@@ -18,19 +18,26 @@ package org.apache.lucene.search;
 
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -180,155 +187,125 @@ public class MultiPhraseQuery extends Query {
     return positions;
   }
 
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    if (termArrays.length == 0) {
+      return new MatchNoDocsQuery("empty MultiPhraseQuery");
+    } else if (termArrays.length == 1) {                 // optimize one-term case
+      Term[] terms = termArrays[0];
+      BooleanQuery.Builder builder = new BooleanQuery.Builder();
+      for (Term term : terms) {
+        builder.add(new TermQuery(term), BooleanClause.Occur.SHOULD);
+      }
+      return builder.build();
+    } else {
+      return super.rewrite(reader);
+    }
+  }
 
-  private class MultiPhraseWeight extends Weight {
-    private final Similarity similarity;
-    private final Similarity.SimWeight stats;
-    private final Map<Term,TermContext> termContexts = new HashMap<>();
-    private final boolean needsScores;
-
-    public MultiPhraseWeight(IndexSearcher searcher, boolean needsScores, float boost)
-      throws IOException {
-      super(MultiPhraseQuery.this);
-      this.needsScores = needsScores;
-      this.similarity = searcher.getSimilarity(needsScores);
-      final IndexReaderContext context = searcher.getTopReaderContext();
-
-      // compute idf
-      ArrayList<TermStatistics> allTermStats = new ArrayList<>();
-      for(final Term[] terms: termArrays) {
-        for (Term term: terms) {
-          TermContext termContext = termContexts.get(term);
-          if (termContext == null) {
-            termContext = TermContext.build(context, term);
-            termContexts.put(term, termContext);
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    final Map<Term, TermContext> termStates = new HashMap<>();
+    return new PhraseWeight(this, field, searcher, needsScores) {
+      @Override
+      protected Similarity.SimWeight getStats(IndexSearcher searcher) throws IOException {
+        final IndexReaderContext context = searcher.getTopReaderContext();
+
+        // compute idf
+        ArrayList<TermStatistics> allTermStats = new ArrayList<>();
+        for(final Term[] terms: termArrays) {
+          for (Term term: terms) {
+            TermContext ts = termStates.get(term);
+            if (ts == null) {
+              ts = TermContext.build(context, term);
+              termStates.put(term, ts);
+            }
+            if (needsScores) {
+              TermStatistics termStatistics = searcher.termStatistics(term, ts);
+              if (termStatistics != null) {
+                allTermStats.add(termStatistics);
+              }
+            }
+            allTermStats.add(searcher.termStatistics(term, ts));
           }
-          allTermStats.add(searcher.termStatistics(term, termContext));
+        }
+        if (allTermStats.isEmpty()) {
+          return null; // none of the terms were found, we won't use sim at all
+        } else {
+          return similarity.computeWeight(
+              boost,
+              searcher.collectionStatistics(field),
+              allTermStats.toArray(new TermStatistics[allTermStats.size()]));
         }
       }
-      stats = similarity.computeWeight(
-          boost,
-          searcher.collectionStatistics(field),
-          allTermStats.toArray(new TermStatistics[allTermStats.size()]));
-    }
 
-    @Override
-    public void extractTerms(Set<Term> terms) {
-      for (final Term[] arr : termArrays) {
-        Collections.addAll(terms, arr);
-      }
-    }
+      @Override
+      protected PhraseMatcher getPhraseMatcher(LeafReaderContext context, boolean exposeOffsets) throws IOException {
+        assert termArrays.length != 0;
+        final LeafReader reader = context.reader();
 
-    @Override
-    public Scorer scorer(LeafReaderContext context) throws IOException {
-      assert termArrays.length != 0;
-      final LeafReader reader = context.reader();
+        PhraseQuery.PostingsAndFreq[] postingsFreqs = new PhraseQuery.PostingsAndFreq[termArrays.length];
 
-      PhraseQuery.PostingsAndFreq[] postingsFreqs = new PhraseQuery.PostingsAndFreq[termArrays.length];
+        final Terms fieldTerms = reader.terms(field);
+        if (fieldTerms == null) {
+          return null;
+        }
 
-      final Terms fieldTerms = reader.terms(field);
-      if (fieldTerms == null) {
-        return null;
-      }
+        // TODO: move this check to createWeight to happen earlier to the user?
+        if (fieldTerms.hasPositions() == false) {
+          throw new IllegalStateException("field \"" + field + "\" was indexed without position data;" +
+              " cannot run MultiPhraseQuery (phrase=" + getQuery() + ")");
+        }
 
-      // TODO: move this check to createWeight to happen earlier to the user?
-      if (fieldTerms.hasPositions() == false) {
-        throw new IllegalStateException("field \"" + field + "\" was indexed without position data;" +
-            " cannot run MultiPhraseQuery (phrase=" + getQuery() + ")");
-      }
+        // Reuse single TermsEnum below:
+        final TermsEnum termsEnum = fieldTerms.iterator();
+        float totalMatchCost = 0;
 
-      // Reuse single TermsEnum below:
-      final TermsEnum termsEnum = fieldTerms.iterator();
-      float totalMatchCost = 0;
+        for (int pos=0; pos<postingsFreqs.length; pos++) {
+          Term[] terms = termArrays[pos];
+          List<PostingsEnum> postings = new ArrayList<>();
 
-      for (int pos=0; pos<postingsFreqs.length; pos++) {
-        Term[] terms = termArrays[pos];
-        List<PostingsEnum> postings = new ArrayList<>();
+          for (Term term : terms) {
+            TermState termState = termStates.get(term).get(context.ord);
+            if (termState != null) {
+              termsEnum.seekExact(term.bytes(), termState);
+              postings.add(termsEnum.postings(null, exposeOffsets ? PostingsEnum.OFFSETS : PostingsEnum.POSITIONS));
+              totalMatchCost += PhraseQuery.termPositionsCost(termsEnum);
+            }
+          }
 
-        for (Term term : terms) {
-          TermState termState = termContexts.get(term).get(context.ord);
-          if (termState != null) {
-            termsEnum.seekExact(term.bytes(), termState);
-            postings.add(termsEnum.postings(null, PostingsEnum.POSITIONS));
-            totalMatchCost += PhraseQuery.termPositionsCost(termsEnum);
+          if (postings.isEmpty()) {
+            return null;
           }
-        }
 
-        if (postings.isEmpty()) {
-          return null;
-        }
+          final PostingsEnum postingsEnum;
+          if (postings.size() == 1) {
+            postingsEnum = postings.get(0);
+          } else {
+            postingsEnum = exposeOffsets ? new UnionFullPostingsEnum(postings) : new UnionPostingsEnum(postings);
+          }
 
-        final PostingsEnum postingsEnum;
-        if (postings.size() == 1) {
-          postingsEnum = postings.get(0);
-        } else {
-          postingsEnum = new UnionPostingsEnum(postings);
+          postingsFreqs[pos] = new PhraseQuery.PostingsAndFreq(postingsEnum, positions[pos], terms);
         }
 
-        postingsFreqs[pos] = new PhraseQuery.PostingsAndFreq(postingsEnum, positions[pos], terms);
-      }
-
-      // sort by increasing docFreq order
-      if (slop == 0) {
-        ArrayUtil.timSort(postingsFreqs);
-      }
+        // sort by increasing docFreq order
+        if (slop == 0) {
+          ArrayUtil.timSort(postingsFreqs);
+          return new ExactPhraseMatcher(postingsFreqs, totalMatchCost);
+        }
+        else {
+          return new SloppyPhraseMatcher(postingsFreqs, slop, totalMatchCost);
+        }
 
-      if (slop == 0) {
-        return new ExactPhraseScorer(this, postingsFreqs,
-                                      similarity.simScorer(stats, context),
-                                      needsScores, totalMatchCost);
-      } else {
-        return new SloppyPhraseScorer(this, postingsFreqs, slop,
-                                        similarity.simScorer(stats, context),
-                                        needsScores, totalMatchCost);
       }
-    }
-
-    @Override
-    public boolean isCacheable(LeafReaderContext ctx) {
-      return true;
-    }
 
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context);
-      if (scorer != null) {
-        int newDoc = scorer.iterator().advance(doc);
-        if (newDoc == doc) {
-          float freq = slop == 0 ? ((ExactPhraseScorer)scorer).freq() : ((SloppyPhraseScorer)scorer).sloppyFreq();
-          SimScorer docScorer = similarity.simScorer(stats, context);
-          Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq);
-          Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
-          return Explanation.match(
-              scoreExplanation.getValue(),
-              "weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:",
-              scoreExplanation);
+      @Override
+      public void extractTerms(Set<Term> terms) {
+        for (final Term[] arr : termArrays) {
+          Collections.addAll(terms, arr);
         }
       }
-
-      return Explanation.noMatch("no matching term");
-    }
-  }
-
-  @Override
-  public Query rewrite(IndexReader reader) throws IOException {
-    if (termArrays.length == 0) {
-      return new MatchNoDocsQuery("empty MultiPhraseQuery");
-    } else if (termArrays.length == 1) {                 // optimize one-term case
-      Term[] terms = termArrays[0];
-      BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      for (Term term : terms) {
-        builder.add(new TermQuery(term), BooleanClause.Occur.SHOULD);
-      }
-      return builder.build();
-    } else {
-      return super.rewrite(reader);
-    }
-  }
-
-  @Override
-  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
-    return new MultiPhraseWeight(searcher, needsScores, boost);
+    };
   }
 
   /** Prints a user-readable version of this query. */
@@ -585,4 +562,90 @@ public class MultiPhraseQuery extends Query {
       }
     }
   }
+
+  static class PostingsAndPosition {
+    final PostingsEnum pe;
+    int pos;
+    int upto;
+
+    PostingsAndPosition(PostingsEnum pe) {
+      this.pe = pe;
+    }
+  }
+
+  // Slower version of UnionPostingsEnum that delegates offsets and positions, for
+  // use by MatchesIterator
+  static class UnionFullPostingsEnum extends UnionPostingsEnum {
+
+    int freq = -1;
+    boolean started = false;
+
+    final PriorityQueue<PostingsAndPosition> posQueue;
+    final Collection<PostingsAndPosition> subs;
+
+    UnionFullPostingsEnum(List<PostingsEnum> subs) {
+      super(subs);
+      this.posQueue = new PriorityQueue<PostingsAndPosition>(subs.size()) {
+        @Override
+        protected boolean lessThan(PostingsAndPosition a, PostingsAndPosition b) {
+          return a.pos < b.pos;
+        }
+      };
+      this.subs = new ArrayList<>();
+      for (PostingsEnum pe : subs) {
+        this.subs.add(new PostingsAndPosition(pe));
+      }
+    }
+
+    @Override
+    public int freq() throws IOException {
+      int doc = docID();
+      if (doc == posQueueDoc) {
+        return freq;
+      }
+      freq = 0;
+      started = false;
+      posQueue.clear();
+      for (PostingsAndPosition pp : subs) {
+        if (pp.pe.docID() == doc) {
+          pp.pos = pp.pe.nextPosition();
+          pp.upto = pp.pe.freq();
+          posQueue.add(pp);
+          freq += pp.upto;
+        }
+      }
+      return freq;
+    }
+
+    @Override
+    public int nextPosition() throws IOException {
+      if (started == false) {
+        started = true;
+        return posQueue.top().pos;
+      }
+      if (posQueue.top().upto == 1) {
+        posQueue.pop();
+        return posQueue.top().pos;
+      }
+      posQueue.top().pos = posQueue.top().pe.nextPosition();
+      posQueue.top().upto--;
+      posQueue.updateTop();
+      return posQueue.top().pos;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return posQueue.top().pe.startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return posQueue.top().pe.endOffset();
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return posQueue.top().pe.getPayload();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/PhraseMatcher.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseMatcher.java b/lucene/core/src/java/org/apache/lucene/search/PhraseMatcher.java
new file mode 100644
index 0000000..9695313
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/PhraseMatcher.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.similarities.Similarity;
+
+/**
+ * Base class for exact and sloppy phrase matching
+ *
+ * To find matches on a document, first advance {@link #approximation} to the
+ * relevant document, then call {@link #reset()}.  Clients can then call
+ * {@link #nextMatch()} to iterate over the matches
+ */
+abstract class PhraseMatcher {
+
+  protected final DocIdSetIterator approximation;
+  private final float matchCost;
+
+  PhraseMatcher(DocIdSetIterator approximation, float matchCost) {
+    assert TwoPhaseIterator.unwrap(approximation) == null;
+    this.approximation = approximation;
+    this.matchCost = matchCost;
+  }
+
+  /**
+   * An upper bound on the number of possible matches on this document
+   */
+  abstract float maxFreq() throws IOException;
+
+  /**
+   * Called after {@link #approximation} has been advanced
+   */
+  public abstract void reset() throws IOException;
+
+  /**
+   * Find the next match on the current document, returning {@code false} if there
+   * are none.
+   */
+  public abstract boolean nextMatch() throws IOException;
+
+  /**
+   * The slop-adjusted weight of the current match
+   *
+   * The sum of the slop-adjusted weights is used as the freq for scoring
+   */
+  abstract float sloppyWeight(Similarity.SimScorer simScorer);
+
+  /**
+   * The start position of the current match
+   */
+  abstract int startPosition();
+
+  /**
+   * The end position of the current match
+   */
+  abstract int endPosition();
+
+  /**
+   * The start offset of the current match
+   */
+  abstract int startOffset() throws IOException;
+
+  /**
+   * The end offset of the current match
+   */
+  abstract int endOffset() throws IOException;
+
+  /**
+   * An estimate of the average cost of finding all matches on a document
+   *
+   * @see TwoPhaseIterator#matchCost()
+   */
+  public float getMatchCost() {
+    return matchCost;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
index 5a70a63..9bd4e53 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
@@ -37,7 +37,6 @@ import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 
@@ -350,121 +349,6 @@ public class PhraseQuery extends Query {
     }
   }
 
-  private class PhraseWeight extends Weight {
-    private final Similarity similarity;
-    private final Similarity.SimWeight stats;
-    private final boolean needsScores;
-    private transient TermContext states[];
-
-    public PhraseWeight(IndexSearcher searcher, boolean needsScores, float boost)
-      throws IOException {
-      super(PhraseQuery.this);
-      final int[] positions = PhraseQuery.this.getPositions();
-      if (positions.length < 2) {
-        throw new IllegalStateException("PhraseWeight does not support less than 2 terms, call rewrite first");
-      } else if (positions[0] != 0) {
-        throw new IllegalStateException("PhraseWeight requires that the first position is 0, call rewrite first");
-      }
-      this.needsScores = needsScores;
-      this.similarity = searcher.getSimilarity(needsScores);
-      final IndexReaderContext context = searcher.getTopReaderContext();
-      states = new TermContext[terms.length];
-      TermStatistics termStats[] = new TermStatistics[terms.length];
-      for (int i = 0; i < terms.length; i++) {
-        final Term term = terms[i];
-        states[i] = TermContext.build(context, term);
-        termStats[i] = searcher.termStatistics(term, states[i]);
-      }
-      stats = similarity.computeWeight(boost, searcher.collectionStatistics(field), termStats);
-    }
-
-    @Override
-    public void extractTerms(Set<Term> queryTerms) {
-      Collections.addAll(queryTerms, terms);
-    }
-
-    @Override
-    public String toString() { return "weight(" + PhraseQuery.this + ")"; }
-
-    @Override
-    public Scorer scorer(LeafReaderContext context) throws IOException {
-      assert terms.length > 0;
-      final LeafReader reader = context.reader();
-      PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.length];
-
-      final Terms fieldTerms = reader.terms(field);
-      if (fieldTerms == null) {
-        return null;
-      }
-
-      if (fieldTerms.hasPositions() == false) {
-        throw new IllegalStateException("field \"" + field + "\" was indexed without position data; cannot run PhraseQuery (phrase=" + getQuery() + ")");
-      }
-
-      // Reuse single TermsEnum below:
-      final TermsEnum te = fieldTerms.iterator();
-      float totalMatchCost = 0;
-      
-      for (int i = 0; i < terms.length; i++) {
-        final Term t = terms[i];
-        final TermState state = states[i].get(context.ord);
-        if (state == null) { /* term doesnt exist in this segment */
-          assert termNotInReader(reader, t): "no termstate found but term exists in reader";
-          return null;
-        }
-        te.seekExact(t.bytes(), state);
-        PostingsEnum postingsEnum = te.postings(null, PostingsEnum.POSITIONS);
-        postingsFreqs[i] = new PostingsAndFreq(postingsEnum, positions[i], t);
-        totalMatchCost += termPositionsCost(te);
-      }
-
-      // sort by increasing docFreq order
-      if (slop == 0) {
-        ArrayUtil.timSort(postingsFreqs);
-      }
-
-      if (slop == 0) {  // optimize exact case
-        return new ExactPhraseScorer(this, postingsFreqs,
-                                      similarity.simScorer(stats, context),
-                                      needsScores, totalMatchCost);
-      } else {
-        return new SloppyPhraseScorer(this, postingsFreqs, slop,
-                                        similarity.simScorer(stats, context),
-                                        needsScores, totalMatchCost);
-      }
-    }
-
-    @Override
-    public boolean isCacheable(LeafReaderContext ctx) {
-      return true;
-    }
-
-    // only called from assert
-    private boolean termNotInReader(LeafReader reader, Term term) throws IOException {
-      return reader.docFreq(term) == 0;
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context);
-      if (scorer != null) {
-        int newDoc = scorer.iterator().advance(doc);
-        if (newDoc == doc) {
-          float freq = slop == 0 ? ((ExactPhraseScorer)scorer).freq() : ((SloppyPhraseScorer)scorer).sloppyFreq();
-          SimScorer docScorer = similarity.simScorer(stats, context);
-          Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq);
-          Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
-          return Explanation.match(
-              scoreExplanation.getValue(),
-              "weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:",
-              scoreExplanation);
-        }
-      }
-      
-      return Explanation.noMatch("no matching term");
-    }
-  }
-
   /** A guess of
    * the average number of simple operations for the initial seek and buffer refill
    * per document for the positions of a term.
@@ -500,10 +384,93 @@ public class PhraseQuery extends Query {
     return TERM_POSNS_SEEK_OPS_PER_DOC + expOccurrencesInMatchingDoc * TERM_OPS_PER_POS;
   }
 
-
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
-    return new PhraseWeight(searcher, needsScores, boost);
+    return new PhraseWeight(this, field, searcher, needsScores) {
+
+      private transient TermContext states[];
+
+      @Override
+      protected Similarity.SimWeight getStats(IndexSearcher searcher) throws IOException {
+        final int[] positions = PhraseQuery.this.getPositions();
+        if (positions.length < 2) {
+          throw new IllegalStateException("PhraseWeight does not support less than 2 terms, call rewrite first");
+        } else if (positions[0] != 0) {
+          throw new IllegalStateException("PhraseWeight requires that the first position is 0, call rewrite first");
+        }
+        final IndexReaderContext context = searcher.getTopReaderContext();
+        states = new TermContext[terms.length];
+        TermStatistics termStats[] = new TermStatistics[terms.length];
+        int termUpTo = 0;
+        for (int i = 0; i < terms.length; i++) {
+          final Term term = terms[i];
+          states[i] = TermContext.build(context, term);
+          if (needsScores) {
+            TermStatistics termStatistics = searcher.termStatistics(term, states[i]);
+            if (termStatistics != null) {
+              termStats[termUpTo++] = termStatistics;
+            }
+          }
+        }
+        if (termUpTo > 0) {
+          return similarity.computeWeight(boost, searcher.collectionStatistics(field), Arrays.copyOf(termStats, termUpTo));
+        } else {
+          return null; // no terms at all, we won't use similarity
+        }
+      }
+
+      @Override
+      protected PhraseMatcher getPhraseMatcher(LeafReaderContext context, boolean exposeOffsets) throws IOException {
+        assert terms.length > 0;
+        final LeafReader reader = context.reader();
+        PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.length];
+
+        final Terms fieldTerms = reader.terms(field);
+        if (fieldTerms == null) {
+          return null;
+        }
+
+        if (fieldTerms.hasPositions() == false) {
+          throw new IllegalStateException("field \"" + field + "\" was indexed without position data; cannot run PhraseQuery (phrase=" + getQuery() + ")");
+        }
+
+        // Reuse single TermsEnum below:
+        final TermsEnum te = fieldTerms.iterator();
+        float totalMatchCost = 0;
+
+        for (int i = 0; i < terms.length; i++) {
+          final Term t = terms[i];
+          final TermState state = states[i].get(context.ord);
+          if (state == null) { /* term doesnt exist in this segment */
+            assert termNotInReader(reader, t): "no termstate found but term exists in reader";
+            return null;
+          }
+          te.seekExact(t.bytes(), state);
+          PostingsEnum postingsEnum = te.postings(null, exposeOffsets ? PostingsEnum.OFFSETS : PostingsEnum.POSITIONS);
+          postingsFreqs[i] = new PostingsAndFreq(postingsEnum, positions[i], t);
+          totalMatchCost += termPositionsCost(te);
+        }
+
+        // sort by increasing docFreq order
+        if (slop == 0) {
+          ArrayUtil.timSort(postingsFreqs);
+          return new ExactPhraseMatcher(postingsFreqs, totalMatchCost);
+        }
+        else {
+          return new SloppyPhraseMatcher(postingsFreqs, slop, totalMatchCost);
+        }
+      }
+
+      @Override
+      public void extractTerms(Set<Term> queryTerms) {
+        Collections.addAll(queryTerms, terms);
+      }
+    };
+  }
+
+  // only called from assert
+  private static boolean termNotInReader(LeafReader reader, Term term) throws IOException {
+    return reader.docFreq(term) == 0;
   }
 
   /** Prints a user-readable version of this query. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java b/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java
new file mode 100644
index 0000000..66e9901
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.similarities.Similarity;
+
+class PhraseScorer extends Scorer {
+
+  final PhraseMatcher matcher;
+  final boolean needsScores;
+  private final Similarity.SimScorer simScorer;
+  final float matchCost;
+
+  private float freq = 0;
+
+  PhraseScorer(Weight weight, PhraseMatcher matcher, boolean needsScores, Similarity.SimScorer simScorer) {
+    super(weight);
+    this.matcher = matcher;
+    this.needsScores = needsScores;
+    this.simScorer = simScorer;
+    this.matchCost = matcher.getMatchCost();
+  }
+
+  @Override
+  public TwoPhaseIterator twoPhaseIterator() {
+    return new TwoPhaseIterator(matcher.approximation) {
+      @Override
+      public boolean matches() throws IOException {
+        matcher.reset();
+        freq = 0;
+        return matcher.nextMatch();
+      }
+
+      @Override
+      public float matchCost() {
+        return matchCost;
+      }
+    };
+  }
+
+  @Override
+  public int docID() {
+    return matcher.approximation.docID();
+  }
+
+  @Override
+  public float score() throws IOException {
+    if (freq == 0) {
+      freq = matcher.sloppyWeight(simScorer);
+      while (matcher.nextMatch()) {
+        freq += matcher.sloppyWeight(simScorer);
+      }
+    }
+    return simScorer.score(docID(), freq);
+  }
+
+  @Override
+  public DocIdSetIterator iterator() {
+    return TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator());
+  }
+
+  @Override
+  public String toString() {
+    return "PhraseScorer(" + weight + ")";
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java b/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java
new file mode 100644
index 0000000..ebc97ff
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.similarities.Similarity;
+
+abstract class PhraseWeight extends Weight {
+
+  final boolean needsScores;
+  final Similarity.SimWeight stats;
+  final Similarity similarity;
+  final String field;
+
+  protected PhraseWeight(Query query, String field, IndexSearcher searcher, boolean needsScores) throws IOException {
+    super(query);
+    this.needsScores = needsScores;
+    this.field = field;
+    this.similarity = searcher.getSimilarity(needsScores);
+    this.stats = getStats(searcher);
+  }
+
+  protected abstract Similarity.SimWeight getStats(IndexSearcher searcher) throws IOException;
+
+  protected abstract PhraseMatcher getPhraseMatcher(LeafReaderContext context, boolean exposeOffsets) throws IOException;
+
+  @Override
+  public Scorer scorer(LeafReaderContext context) throws IOException {
+    PhraseMatcher matcher = getPhraseMatcher(context, false);
+    if (matcher == null)
+      return null;
+    Similarity.SimScorer simScorer = similarity.simScorer(stats, context);
+    return new PhraseScorer(this, matcher, needsScores, simScorer);
+  }
+
+  @Override
+  public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+    PhraseMatcher matcher = getPhraseMatcher(context, false);
+    if (matcher == null || matcher.approximation.advance(doc) != doc) {
+      return Explanation.noMatch("no matching terms");
+    }
+    matcher.reset();
+    if (matcher.nextMatch() == false) {
+      return Explanation.noMatch("no matching phrase");
+    }
+    Similarity.SimScorer simScorer = similarity.simScorer(stats, context);
+    float freq = matcher.sloppyWeight(simScorer);
+    while (matcher.nextMatch()) {
+      freq += matcher.sloppyWeight(simScorer);
+    }
+    Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq);
+    Explanation scoreExplanation = simScorer.explain(doc, freqExplanation);
+    return Explanation.match(
+        scoreExplanation.getValue(),
+        "weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:",
+        scoreExplanation);
+  }
+
+  @Override
+  public Matches matches(LeafReaderContext context, int doc) throws IOException {
+    return Matches.forField(field, () -> {
+      PhraseMatcher matcher = getPhraseMatcher(context, true);
+      if (matcher == null || matcher.approximation.advance(doc) != doc) {
+        return null;
+      }
+      matcher.reset();
+      if (matcher.nextMatch() == false) {
+        return null;
+      }
+      return new MatchesIterator() {
+        boolean started = false;
+        @Override
+        public boolean next() throws IOException {
+          if (started == false) {
+            return started = true;
+          }
+          return matcher.nextMatch();
+        }
+
+        @Override
+        public int startPosition() {
+          return matcher.startPosition();
+        }
+
+        @Override
+        public int endPosition() {
+          return matcher.endPosition();
+        }
+
+        @Override
+        public int startOffset() throws IOException {
+          return matcher.startOffset();
+        }
+
+        @Override
+        public int endOffset() throws IOException {
+          return matcher.endOffset();
+        }
+      };
+    });
+  }
+
+  @Override
+  public boolean isCacheable(LeafReaderContext ctx) {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67056328/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseMatcher.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseMatcher.java b/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseMatcher.java
new file mode 100644
index 0000000..75a6ef3
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseMatcher.java
@@ -0,0 +1,597 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * Find all slop-valid position-combinations (matches)
+ * encountered while traversing/hopping the PhrasePositions.
+ * <br> The sloppy frequency contribution of a match depends on the distance:
+ * <br> - highest freq for distance=0 (exact match).
+ * <br> - freq gets lower as distance gets higher.
+ * <br>Example: for query "a b"~2, a document "x a b a y" can be matched twice:
+ * once for "a b" (distance=0), and once for "b a" (distance=2).
+ * <br>Possibly not all valid combinations are encountered, because for efficiency
+ * we always propagate the least PhrasePosition. This allows to base on
+ * PriorityQueue and move forward faster.
+ * As result, for example, document "a b c b a"
+ * would score differently for queries "a b c"~4 and "c b a"~4, although
+ * they really are equivalent.
+ * Similarly, for doc "a b c b a f g", query "c b"~2
+ * would get same score as "g f"~2, although "c b"~2 could be matched twice.
+ * We may want to fix this in the future (currently not, for performance reasons).
+ */
+final class SloppyPhraseMatcher extends PhraseMatcher {
+
+  private final PhrasePositions[] phrasePositions;
+
+  private final int slop;
+  private final int numPostings;
+  private final PhraseQueue pq; // for advancing min position
+
+  private int end; // current largest phrase position
+
+  private int leadPosition;
+  private int leadOffset;
+  private int currentEndPostings;
+  private int advanceEndPostings;
+
+  private boolean hasRpts; // flag indicating that there are repetitions (as checked in first candidate doc)
+  private boolean checkedRpts; // flag to only check for repetitions in first candidate doc
+  private boolean hasMultiTermRpts; //  
+  private PhrasePositions[][] rptGroups; // in each group are PPs that repeats each other (i.e. same term), sorted by (query) offset 
+  private PhrasePositions[] rptStack; // temporary stack for switching colliding repeating pps
+
+  private boolean positioned;
+  private int matchLength;
+
+  SloppyPhraseMatcher(PhraseQuery.PostingsAndFreq[] postings, int slop, float matchCost) {
+    super(approximation(postings), matchCost);
+    this.slop = slop;
+    this.numPostings = postings.length;
+    pq = new PhraseQueue(postings.length);
+    phrasePositions = new PhrasePositions[postings.length];
+    for (int i = 0; i < postings.length; ++i) {
+      phrasePositions[i] = new PhrasePositions(postings[i].postings, postings[i].position, i, postings[i].terms);
+    }
+  }
+
+  private static DocIdSetIterator approximation(PhraseQuery.PostingsAndFreq[] postings) {
+    List<DocIdSetIterator> iterators = new ArrayList<>();
+    for (PhraseQuery.PostingsAndFreq posting : postings) {
+      iterators.add(posting.postings);
+    }
+    return ConjunctionDISI.intersectIterators(iterators);
+  }
+
+  @Override
+  float maxFreq() throws IOException {
+    // every term position in each postings list can be at the head of at most
+    // one matching phrase, so the maximum possible phrase freq is the sum of
+    // the freqs of the postings lists.
+    float maxFreq = 0;
+    for (PhrasePositions phrasePosition : phrasePositions) {
+      maxFreq += phrasePosition.postings.freq();
+    }
+    return maxFreq;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    this.positioned = initPhrasePositions();
+    this.matchLength = Integer.MAX_VALUE;
+    this.leadPosition = Integer.MAX_VALUE;
+  }
+
+  @Override
+  float sloppyWeight(Similarity.SimScorer simScorer) {
+    return simScorer.computeSlopFactor(matchLength);
+  }
+
+  @Override
+  public boolean nextMatch() throws IOException {
+    if (!positioned) {
+      return false;
+    }
+    PhrasePositions pp = pq.pop();
+    assert pp != null;  // if the pq is empty, then positioned == false
+    leadPosition = pp.position + pp.offset;
+    leadOffset = pp.postings.startOffset();
+    currentEndPostings = advanceEndPostings;
+    matchLength = end - pp.position;
+    int next = pq.top().position; 
+    while (advancePP(pp)) {
+      if (hasRpts && !advanceRpts(pp)) {
+        break; // pps exhausted
+      }
+      if (pp.position > next) { // done minimizing current match-length
+        pq.add(pp);
+        if (matchLength <= slop) {
+          return true;
+        }
+        pp = pq.pop();
+        next = pq.top().position;
+        matchLength = end - pp.position;
+      } else {
+        int matchLength2 = end - pp.position;
+        if (matchLength2 < matchLength) {
+          matchLength = matchLength2;
+        }
+      }
+      leadPosition = pp.position + pp.offset;
+      leadOffset = pp.postings.startOffset();
+      currentEndPostings = advanceEndPostings;
+    }
+    positioned = false;
+    return matchLength <= slop;
+  }
+
+  @Override
+  public int startPosition() {
+    // when a match is detected, the top postings is advanced until it has moved
+    // beyond its successor, to ensure that the match is of minimal width.  This
+    // means that we need to record the lead position before it is advanced.
+    // However, the priority queue doesn't guarantee that the top postings is in fact the
+    // earliest in the list, so we need to cycle through all terms to check.
+    // this is slow, but Matches is slow anyway...
+    for (PhrasePositions pp : phrasePositions) {
+      leadPosition = Math.min(leadPosition, pp.position + pp.offset);
+    }
+    return leadPosition;
+  }
+
+  @Override
+  public int endPosition() {
+    return phrasePositions[currentEndPostings].position + phrasePositions[currentEndPostings].offset;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    // when a match is detected, the top postings is advanced until it has moved
+    // beyond its successor, to ensure that the match is of minimal width.  This
+    // means that we need to record the lead offset before it is advanced.
+    // However, the priority queue doesn't guarantee that the top postings is in fact the
+    // earliest in the list, so we need to cycle through all terms to check
+    // this is slow, but Matches is slow anyway...
+    for (PhrasePositions pp : phrasePositions) {
+      leadOffset = Math.min(leadOffset, pp.postings.startOffset());
+    }
+    return leadOffset;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return phrasePositions[currentEndPostings].postings.endOffset();
+  }
+
+  /** advance a PhrasePosition and update 'end', return false if exhausted */
+  private boolean advancePP(PhrasePositions pp) throws IOException {
+    if (!pp.nextPosition()) {
+      return false;
+    }
+    if (pp.position > end) {
+      end = pp.position;
+      advanceEndPostings = pp.ord;
+    }
+    if (pp.position == end) {
+      if (pp.ord > advanceEndPostings) {
+        advanceEndPostings = pp.ord;
+      }
+    }
+    return true;
+  }
+  
+  /** pp was just advanced. If that caused a repeater collision, resolve by advancing the lesser
+   * of the two colliding pps. Note that there can only be one collision, as by the initialization
+   * there were no collisions before pp was advanced.  */
+  private boolean advanceRpts(PhrasePositions pp) throws IOException {
+    if (pp.rptGroup < 0) {
+      return true; // not a repeater
+    }
+    PhrasePositions[] rg = rptGroups[pp.rptGroup];
+    FixedBitSet bits = new FixedBitSet(rg.length); // for re-queuing after collisions are resolved
+    int k0 = pp.rptInd;
+    int k;
+    while((k=collide(pp)) >= 0) {
+      pp = lesser(pp, rg[k]); // always advance the lesser of the (only) two colliding pps
+      if (!advancePP(pp)) {
+        return false; // exhausted
+      }
+      if (k != k0) { // careful: mark only those currently in the queue
+        bits = FixedBitSet.ensureCapacity(bits, k);
+        bits.set(k); // mark that pp2 need to be re-queued
+      }
+    }
+    // collisions resolved, now re-queue
+    // empty (partially) the queue until seeing all pps advanced for resolving collisions
+    int n = 0;
+    // TODO would be good if we can avoid calling cardinality() in each iteration!
+    int numBits = bits.length(); // larges bit we set
+    while (bits.cardinality() > 0) {
+      PhrasePositions pp2 = pq.pop();
+      rptStack[n++] = pp2;
+      if (pp2.rptGroup >= 0 
+          && pp2.rptInd < numBits  // this bit may not have been set
+          && bits.get(pp2.rptInd)) {
+        bits.clear(pp2.rptInd);
+      }
+    }
+    // add back to queue
+    for (int i=n-1; i>=0; i--) {
+      pq.add(rptStack[i]);
+    }
+    return true;
+  }
+
+  /** compare two pps, but only by position and offset */
+  private PhrasePositions lesser(PhrasePositions pp, PhrasePositions pp2) {
+    if (pp.position < pp2.position ||
+        (pp.position == pp2.position && pp.offset < pp2.offset)) {
+      return pp;
+    }
+    return pp2;
+  }
+
+  /** index of a pp2 colliding with pp, or -1 if none */
+  private int collide(PhrasePositions pp) {
+    int tpPos = tpPos(pp);
+    PhrasePositions[] rg = rptGroups[pp.rptGroup];
+    for (int i=0; i<rg.length; i++) {
+      PhrasePositions pp2 = rg[i];
+      if (pp2 != pp && tpPos(pp2) == tpPos) {
+        return pp2.rptInd;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Initialize PhrasePositions in place.
+   * A one time initialization for this scorer (on first doc matching all terms):
+   * <ul>
+   *  <li>Check if there are repetitions
+   *  <li>If there are, find groups of repetitions.
+   * </ul>
+   * Examples:
+   * <ol>
+   *  <li>no repetitions: <b>"ho my"~2</b>
+   *  <li>repetitions: <b>"ho my my"~2</b>
+   *  <li>repetitions: <b>"my ho my"~2</b>
+   * </ol>
+   * @return false if PPs are exhausted (and so current doc will not be a match) 
+   */
+  private boolean initPhrasePositions() throws IOException {
+    end = Integer.MIN_VALUE;
+    if (!checkedRpts) {
+      return initFirstTime();
+    }
+    if (!hasRpts) {
+      initSimple();
+      return true; // PPs available
+    }
+    return initComplex();
+  }
+  
+  /** no repeats: simplest case, and most common. It is important to keep this piece of the code simple and efficient */
+  private void initSimple() throws IOException {
+    //System.err.println("initSimple: doc: "+min.doc);
+    pq.clear();
+    // position pps and build queue from list
+    for (PhrasePositions pp : phrasePositions) {
+      pp.firstPosition();
+      if (pp.position > end) {
+        end = pp.position;
+        advanceEndPostings = pp.ord;
+      }
+      if (pp.position == end) {
+        if (pp.ord > advanceEndPostings) {
+          advanceEndPostings = pp.ord;
+        }
+      }
+      pq.add(pp);
+    }
+  }
+  
+  /** with repeats: not so simple. */
+  private boolean initComplex() throws IOException {
+    //System.err.println("initComplex: doc: "+min.doc);
+    placeFirstPositions();
+    if (!advanceRepeatGroups()) {
+      return false; // PPs exhausted
+    }
+    fillQueue();
+    return true; // PPs available
+  }
+
+  /** move all PPs to their first position */
+  private void placeFirstPositions() throws IOException {
+    for (PhrasePositions pp : phrasePositions) {
+      pp.firstPosition();
+    }
+  }
+
+  /** Fill the queue (all pps are already placed */
+  private void fillQueue() {
+    pq.clear();
+    for (PhrasePositions pp : phrasePositions) {  // iterate cyclic list: done once handled max
+      if (pp.position > end) {
+        end = pp.position;
+        advanceEndPostings = pp.ord;
+      }
+      if (pp.position == end) {
+        if (pp.ord > advanceEndPostings) {
+          advanceEndPostings = pp.ord;
+        }
+      }
+      pq.add(pp);
+    }
+  }
+
+  /** At initialization (each doc), each repetition group is sorted by (query) offset.
+   * This provides the start condition: no collisions.
+   * <p>Case 1: no multi-term repeats<br>
+   * It is sufficient to advance each pp in the group by one less than its group index.
+   * So lesser pp is not advanced, 2nd one advance once, 3rd one advanced twice, etc.
+   * <p>Case 2: multi-term repeats<br>
+   * 
+   * @return false if PPs are exhausted. 
+   */
+  private boolean advanceRepeatGroups() throws IOException {
+    for (PhrasePositions[] rg: rptGroups) { 
+      if (hasMultiTermRpts) {
+        // more involved, some may not collide
+        int incr;
+        for (int i=0; i<rg.length; i+=incr) {
+          incr = 1;
+          PhrasePositions pp = rg[i];
+          int k;
+          while((k=collide(pp)) >= 0) {
+            PhrasePositions pp2 = lesser(pp, rg[k]);
+            if (!advancePP(pp2)) {  // at initialization always advance pp with higher offset
+              return false; // exhausted
+            }
+            if (pp2.rptInd < i) { // should not happen?
+              incr = 0;
+              break;
+            }
+          }
+        }
+      } else {
+        // simpler, we know exactly how much to advance
+        for (int j=1; j<rg.length; j++) {
+          for (int k=0; k<j; k++) {
+            if (!rg[j].nextPosition()) {
+              return false; // PPs exhausted
+            }
+          }
+        }
+      }
+    }
+    return true; // PPs available
+  }
+  
+  /** initialize with checking for repeats. Heavy work, but done only for the first candidate doc.<p>
+   * If there are repetitions, check if multi-term postings (MTP) are involved.<p>
+   * Without MTP, once PPs are placed in the first candidate doc, repeats (and groups) are visible.<br>
+   * With MTP, a more complex check is needed, up-front, as there may be "hidden collisions".<br>
+   * For example P1 has {A,B}, P1 has {B,C}, and the first doc is: "A C B". At start, P1 would point
+   * to "A", p2 to "C", and it will not be identified that P1 and P2 are repetitions of each other.<p>
+   * The more complex initialization has two parts:<br>
+   * (1) identification of repetition groups.<br>
+   * (2) advancing repeat groups at the start of the doc.<br>
+   * For (1), a possible solution is to just create a single repetition group, 
+   * made of all repeating pps. But this would slow down the check for collisions, 
+   * as all pps would need to be checked. Instead, we compute "connected regions" 
+   * on the bipartite graph of postings and terms.  
+   */
+  private boolean initFirstTime() throws IOException {
+    //System.err.println("initFirstTime: doc: "+min.doc);
+    checkedRpts = true;
+    placeFirstPositions();
+
+    LinkedHashMap<Term,Integer> rptTerms = repeatingTerms(); 
+    hasRpts = !rptTerms.isEmpty();
+
+    if (hasRpts) {
+      rptStack = new PhrasePositions[numPostings]; // needed with repetitions
+      ArrayList<ArrayList<PhrasePositions>> rgs = gatherRptGroups(rptTerms);
+      sortRptGroups(rgs);
+      if (!advanceRepeatGroups()) {
+        return false; // PPs exhausted
+      }
+    }
+    
+    fillQueue();
+    return true; // PPs available
+  }
+
+  /** sort each repetition group by (query) offset. 
+   * Done only once (at first doc) and allows to initialize faster for each doc. */
+  private void sortRptGroups(ArrayList<ArrayList<PhrasePositions>> rgs) {
+    rptGroups = new PhrasePositions[rgs.size()][];
+    Comparator<PhrasePositions> cmprtr = new Comparator<PhrasePositions>() {
+      @Override
+      public int compare(PhrasePositions pp1, PhrasePositions pp2) {
+        return pp1.offset - pp2.offset;
+      }
+    };
+    for (int i=0; i<rptGroups.length; i++) {
+      PhrasePositions[] rg = rgs.get(i).toArray(new PhrasePositions[0]);
+      Arrays.sort(rg, cmprtr);
+      rptGroups[i] = rg;
+      for (int j=0; j<rg.length; j++) {
+        rg[j].rptInd = j; // we use this index for efficient re-queuing
+      }
+    }
+  }
+
+  /** Detect repetition groups. Done once - for first doc */
+  private ArrayList<ArrayList<PhrasePositions>> gatherRptGroups(LinkedHashMap<Term,Integer> rptTerms) throws IOException {
+    PhrasePositions[] rpp = repeatingPPs(rptTerms); 
+    ArrayList<ArrayList<PhrasePositions>> res = new ArrayList<>();
+    if (!hasMultiTermRpts) {
+      // simpler - no multi-terms - can base on positions in first doc
+      for (int i=0; i<rpp.length; i++) {
+        PhrasePositions pp = rpp[i];
+        if (pp.rptGroup >=0) continue; // already marked as a repetition
+        int tpPos = tpPos(pp);
+        for (int j=i+1; j<rpp.length; j++) {
+          PhrasePositions pp2 = rpp[j];
+          if (
+              pp2.rptGroup >=0        // already marked as a repetition
+              || pp2.offset == pp.offset // not a repetition: two PPs are originally in same offset in the query! 
+              || tpPos(pp2) != tpPos) {  // not a repetition
+            continue; 
+          }
+          // a repetition
+          int g = pp.rptGroup;
+          if (g < 0) {
+            g = res.size();
+            pp.rptGroup = g;  
+            ArrayList<PhrasePositions> rl = new ArrayList<>(2);
+            rl.add(pp);
+            res.add(rl); 
+          }
+          pp2.rptGroup = g;
+          res.get(g).add(pp2);
+        }
+      }
+    } else {
+      // more involved - has multi-terms
+      ArrayList<HashSet<PhrasePositions>> tmp = new ArrayList<>();
+      ArrayList<FixedBitSet> bb = ppTermsBitSets(rpp, rptTerms);
+      unionTermGroups(bb);
+      HashMap<Term,Integer> tg = termGroups(rptTerms, bb);
+      HashSet<Integer> distinctGroupIDs = new HashSet<>(tg.values());
+      for (int i=0; i<distinctGroupIDs.size(); i++) {
+        tmp.add(new HashSet<PhrasePositions>());
+      }
+      for (PhrasePositions pp : rpp) {
+        for (Term t: pp.terms) {
+          if (rptTerms.containsKey(t)) {
+            int g = tg.get(t);
+            tmp.get(g).add(pp);
+            assert pp.rptGroup==-1 || pp.rptGroup==g;  
+            pp.rptGroup = g;
+          }
+        }
+      }
+      for (HashSet<PhrasePositions> hs : tmp) {
+        res.add(new ArrayList<>(hs));
+      }
+    }
+    return res;
+  }
+
+  /** Actual position in doc of a PhrasePosition, relies on that position = tpPos - offset) */
+  private final int tpPos(PhrasePositions pp) {
+    return pp.position + pp.offset;
+  }
+
+  /** find repeating terms and assign them ordinal values */
+  private LinkedHashMap<Term,Integer> repeatingTerms() {
+    LinkedHashMap<Term,Integer> tord = new LinkedHashMap<>();
+    HashMap<Term,Integer> tcnt = new HashMap<>();
+    for (PhrasePositions pp : phrasePositions) {
+      for (Term t : pp.terms) {
+        Integer cnt0 = tcnt.get(t);
+        Integer cnt = cnt0==null ? new Integer(1) : new Integer(1+cnt0.intValue());
+        tcnt.put(t, cnt);
+        if (cnt==2) {
+          tord.put(t,tord.size());
+        }
+      }
+    }
+    return tord;
+  }
+
+  /** find repeating pps, and for each, if has multi-terms, update this.hasMultiTermRpts */
+  private PhrasePositions[] repeatingPPs(HashMap<Term,Integer> rptTerms) {
+    ArrayList<PhrasePositions> rp = new ArrayList<>();
+    for (PhrasePositions pp : phrasePositions) {
+      for (Term t : pp.terms) {
+        if (rptTerms.containsKey(t)) {
+          rp.add(pp);
+          hasMultiTermRpts |= (pp.terms.length > 1);
+          break;
+        }
+      }
+    }
+    return rp.toArray(new PhrasePositions[0]);
+  }
+  
+  /** bit-sets - for each repeating pp, for each of its repeating terms, the term ordinal values is set */
+  private ArrayList<FixedBitSet> ppTermsBitSets(PhrasePositions[] rpp, HashMap<Term,Integer> tord) {
+    ArrayList<FixedBitSet> bb = new ArrayList<>(rpp.length);
+    for (PhrasePositions pp : rpp) {
+      FixedBitSet b = new FixedBitSet(tord.size());
+      Integer ord;
+      for (Term t: pp.terms) {
+        if ((ord=tord.get(t))!=null) {
+          b.set(ord);
+        }
+      }
+      bb.add(b);
+    }
+    return bb;
+  }
+  
+  /** union (term group) bit-sets until they are disjoint (O(n^^2)), and each group have different terms */
+  private void unionTermGroups(ArrayList<FixedBitSet> bb) {
+    int incr;
+    for (int i=0; i<bb.size()-1; i+=incr) {
+      incr = 1;
+      int j = i+1;
+      while (j<bb.size()) {
+        if (bb.get(i).intersects(bb.get(j))) {
+          bb.get(i).or(bb.get(j));
+          bb.remove(j);
+          incr = 0;
+        } else {
+          ++j;
+        }
+      }
+    }
+  }
+  
+  /** map each term to the single group that contains it */ 
+  private HashMap<Term,Integer> termGroups(LinkedHashMap<Term,Integer> tord, ArrayList<FixedBitSet> bb) throws IOException {
+    HashMap<Term,Integer> tg = new HashMap<>();
+    Term[] t = tord.keySet().toArray(new Term[0]);
+    for (int i=0; i<bb.size(); i++) { // i is the group no.
+      FixedBitSet bits = bb.get(i);
+      for (int ord = bits.nextSetBit(0); ord != DocIdSetIterator.NO_MORE_DOCS; ord = ord + 1 >= bits.length() ? DocIdSetIterator.NO_MORE_DOCS : bits.nextSetBit(ord + 1)) {
+        tg.put(t[ord],i);
+      }
+    }
+    return tg;
+  }
+
+}